You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/10/30 11:44:29 UTC

[01/19] ignite git commit: ignite-950-new WIP

Repository: ignite
Updated Branches:
  refs/heads/ignite-950-new e33fa6342 -> 35b6d61fa


http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 4320df5..39d581f 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -347,7 +347,7 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         assert g != null;
 
         return g.portables();


[12/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index 2140bee..d6b526d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.portable.builder.PortableLazyValue;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -185,7 +185,7 @@ public class PortableUtils {
      * @param writer W
      * @param val Value.
      */
-    public static void writePlainObject(PortableWriterExImpl writer, Object val) {
+    public static void writePlainObject(IgniteObjectWriterExImpl writer, Object val) {
         Byte flag = PLAIN_CLASS_TO_FLAG.get(val.getClass());
 
         if (flag == null)
@@ -436,7 +436,7 @@ public class PortableUtils {
     public static boolean isPortableType(Class<?> cls) {
         assert cls != null;
 
-        return PortableObject.class.isAssignableFrom(cls) ||
+        return IgniteObject.class.isAssignableFrom(cls) ||
             PORTABLE_CLS.contains(cls) ||
             cls.isEnum() ||
             (cls.isArray() && cls.getComponentType().isEnum());
@@ -485,6 +485,6 @@ public class PortableUtils {
      */
     public static void checkProtocolVersion(byte protoVer) {
         if (PROTO_VER != protoVer)
-            throw new PortableException("Unsupported protocol version: " + protoVer);
+            throw new IgniteObjectException("Unsupported protocol version: " + protoVer);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
deleted file mode 100644
index a8a4295..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ /dev/null
@@ -1,1892 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
-import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableWriter;
-import org.jetbrains.annotations.Nullable;
-
-import java.io.IOException;
-import java.io.ObjectOutput;
-import java.lang.reflect.InvocationTargetException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.IdentityHashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR;
-
- /**
- * Portable writer implementation.
- */
-public class PortableWriterExImpl implements PortableWriter, PortableRawWriterEx, ObjectOutput {
-    /** Length: integer. */
-    private static final int LEN_INT = 4;
-
-    /** */
-    private static final int INIT_CAP = 1024;
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final WriterContext wCtx;
-
-    /** */
-    private final int start;
-
-    /** */
-    private int mark;
-
-    /** */
-    private Class<?> cls;
-
-    /** */
-    private int typeId;
-
-    /** */
-    private boolean allowFields = true;
-
-    /** */
-    private boolean metaEnabled;
-
-    /** */
-    private int metaHashSum;
-
-    /**
-     * @param ctx Context.
-     * @param off Start offset.
-     */
-    PortableWriterExImpl(PortableContext ctx, int off) {
-        this.ctx = ctx;
-
-        PortableOutputStream out = new PortableHeapOutputStream(off + INIT_CAP);
-
-        out.position(off);
-
-        wCtx = new WriterContext(out, null);
-
-        start = off;
-    }
-
-    /**
-     * @param ctx Context.
-     * @param out Output stream.
-     * @param off Start offset.
-     */
-    PortableWriterExImpl(PortableContext ctx, PortableOutputStream out, int off) {
-        this.ctx = ctx;
-
-        wCtx = new WriterContext(out, null);
-
-        start = off;
-    }
-
-    /**
-     * @param ctx Context.
-     * @param off Start offset.
-     * @param typeId Type ID.
-     */
-    public PortableWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
-        this(ctx, off);
-
-        this.typeId = typeId;
-
-        this.metaEnabled = metaEnabled;
-    }
-
-    /**
-     * @param ctx Context.
-     * @param wCtx Writer context.
-     */
-    private PortableWriterExImpl(PortableContext ctx, WriterContext wCtx) {
-        this.ctx = ctx;
-        this.wCtx = wCtx;
-
-        start = wCtx.out.position();
-    }
-
-    /**
-     * Close the writer releasing resources if necessary.
-     */
-    @Override public void close() {
-        wCtx.out.close();
-    }
-
-    /**
-     * @return Meta data hash sum or {@code null} if meta data is disabled.
-     */
-    @Nullable Integer metaDataHashSum() {
-        return metaEnabled ? metaHashSum : null;
-    }
-
-    /**
-     * @param obj Object.
-     * @param detached Detached or not.
-     * @throws PortableException In case of error.
-     */
-    void marshal(Object obj, boolean detached) throws PortableException {
-        assert obj != null;
-
-        cls = obj.getClass();
-
-        PortableClassDescriptor desc = ctx.descriptorForClass(cls);
-
-        if (desc == null)
-            throw new PortableException("Object is not portable: [class=" + cls + ']');
-
-        if (desc.excluded()) {
-            doWriteByte(NULL);
-            return;
-        }
-
-        if (desc.useOptimizedMarshaller()) {
-            writeByte(OPTM_MARSH);
-
-            try {
-                byte[] arr = ctx.optimizedMarsh().marshal(obj);
-
-                writeInt(arr.length);
-
-                write(arr);
-            }
-            catch (IgniteCheckedException e) {
-                throw new PortableException("Failed to marshal object with optimized marshaller: " + obj, e);
-            }
-
-            return;
-        }
-
-        if (desc.getWriteReplaceMethod() != null) {
-            Object replace;
-
-            try {
-                replace = desc.getWriteReplaceMethod().invoke(obj);
-            }
-            catch (IllegalAccessException e) {
-                throw new RuntimeException(e);
-            }
-            catch (InvocationTargetException e) {
-                if (e.getTargetException() instanceof PortableException)
-                    throw (PortableException)e.getTargetException();
-
-                throw new PortableException("Failed to execute writeReplace() method on " + obj, e);
-            }
-
-            if (replace == null) {
-                doWriteByte(NULL);
-                return;
-            }
-
-            if (cls != replace.getClass()) {
-                cls = replace.getClass();
-
-                desc = ctx.descriptorForClass(cls);
-
-                if (desc == null)
-                    throw new PortableException("Object is not portable: [class=" + cls + ']');
-            }
-
-            obj = replace;
-        }
-
-        typeId = desc.typeId();
-
-        metaEnabled = ctx.isMetaDataEnabled(typeId);
-
-        if (detached)
-            wCtx.resetHandles();
-
-        desc.write(obj, this);
-    }
-
-    /**
-     * @param obj Object.
-     * @return Handle.
-     */
-    int handle(Object obj) {
-        assert obj != null;
-
-        return wCtx.handle(obj);
-    }
-
-    /**
-     * @return Array.
-     */
-    public byte[] array() {
-        return wCtx.out.arrayCopy();
-    }
-
-    /**
-     * @return Output stream.
-     */
-    public PortableOutputStream outputStream() {
-        return wCtx.out;
-    }
-
-    /**
-     * @return Stream current position.
-     */
-    int position() {
-        return wCtx.out.position();
-    }
-
-    /**
-     * Sets new position.
-     *
-     * @param pos Position.
-     */
-    void position(int pos) {
-        wCtx.out.position(pos);
-    }
-
-     /**
-     * @param bytes Number of bytes to reserve.
-     * @return Offset.
-     */
-    public int reserve(int bytes) {
-        int pos = wCtx.out.position();
-
-        wCtx.out.position(pos + bytes);
-
-        return pos;
-    }
-
-    /**
-     * @param bytes Number of bytes to reserve.
-     * @return Offset.
-     */
-    public int reserveAndMark(int bytes) {
-        int off0 = reserve(bytes);
-
-        mark = wCtx.out.position();
-
-        return off0;
-    }
-
-    /**
-     * @param off Offset.
-     */
-    public void writeDelta(int off) {
-        wCtx.out.writeInt(off, wCtx.out.position() - mark);
-    }
-
-    /**
-     *
-     */
-    public void writeLength() {
-        wCtx.out.writeInt(start + TOTAL_LEN_POS, wCtx.out.position() - start);
-    }
-
-    /**
-     *
-     */
-    public void writeRawOffsetIfNeeded() {
-        if (allowFields)
-            wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
-    }
-
-    /**
-     * @param val Byte array.
-     */
-    public void write(byte[] val) {
-        assert val != null;
-
-        wCtx.out.writeByteArray(val);
-    }
-
-    /**
-     * @param val Byte array.
-     * @param off Offset.
-     * @param len Length.
-     */
-    public void write(byte[] val, int off, int len) {
-        assert val != null;
-
-        wCtx.out.write(val, off, len);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteByte(byte val) {
-        wCtx.out.writeByte(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteShort(short val) {
-        wCtx.out.writeShort(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteInt(int val) {
-        wCtx.out.writeInt(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteLong(long val) {
-        wCtx.out.writeLong(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteFloat(float val) {
-        wCtx.out.writeFloat(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteDouble(double val) {
-        wCtx.out.writeDouble(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteChar(char val) {
-        wCtx.out.writeChar(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    public void doWriteBoolean(boolean val) {
-        wCtx.out.writeBoolean(val);
-    }
-
-    /**
-     * @param val String value.
-     */
-    public void doWriteDecimal(@Nullable BigDecimal val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(DECIMAL);
-
-            BigInteger intVal = val.unscaledValue();
-
-            if (intVal.signum() == -1) {
-                intVal = intVal.negate();
-
-                wCtx.out.writeInt(val.scale() | 0x80000000);
-            }
-            else
-                wCtx.out.writeInt(val.scale());
-
-            byte[] vals = intVal.toByteArray();
-
-            wCtx.out.writeInt(vals.length);
-            wCtx.out.writeByteArray(vals);
-        }
-    }
-
-    /**
-     * @param val String value.
-     */
-    public void doWriteString(@Nullable String val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(STRING);
-
-            if (ctx.isConvertString()) {
-                doWriteBoolean(true);
-
-                byte[] strArr = val.getBytes(UTF_8);
-
-                doWriteInt(strArr.length);
-
-                wCtx.out.writeByteArray(strArr);
-            }
-            else {
-                doWriteBoolean(false);
-
-                char[] strArr = val.toCharArray();
-
-                doWriteInt(strArr.length);
-
-                wCtx.out.writeCharArray(strArr);
-            }
-        }
-    }
-
-    /**
-     * @param uuid UUID.
-     */
-    public void doWriteUuid(@Nullable UUID uuid) {
-        if (uuid == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(UUID);
-            doWriteLong(uuid.getMostSignificantBits());
-            doWriteLong(uuid.getLeastSignificantBits());
-        }
-    }
-
-    /**
-     * @param date Date.
-     */
-    public void doWriteDate(@Nullable Date date) {
-        if (date == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(DATE);
-            doWriteLong(date.getTime());
-        }
-    }
-
-     /**
-      * @param ts Timestamp.
-      */
-     public void doWriteTimestamp(@Nullable Timestamp ts) {
-         if (ts== null)
-             doWriteByte(NULL);
-         else {
-             doWriteByte(TIMESTAMP);
-             doWriteLong(ts.getTime());
-             doWriteInt(ts.getNanos() % 1000000);
-         }
-     }
-
-    /**
-     * @param obj Object.
-     * @param detached Detached or not.
-     * @throws PortableException In case of error.
-     */
-    public void doWriteObject(@Nullable Object obj, boolean detached) throws PortableException {
-        if (obj == null)
-            doWriteByte(NULL);
-        else {
-            WriterContext wCtx = detached ? new WriterContext(this.wCtx.out, this.wCtx.handles) : this.wCtx;
-
-            PortableWriterExImpl writer = new PortableWriterExImpl(ctx, wCtx);
-
-            writer.marshal(obj, detached);
-
-            if (detached)
-                this.wCtx.out = wCtx.out;
-        }
-    }
-
-    /**
-     * @param val Byte array.
-     */
-    void doWriteByteArray(@Nullable byte[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(BYTE_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeByteArray(val);
-        }
-    }
-
-    /**
-     * @param val Short array.
-     */
-    void doWriteShortArray(@Nullable short[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(SHORT_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeShortArray(val);
-        }
-    }
-
-    /**
-     * @param val Integer array.
-     */
-    void doWriteIntArray(@Nullable int[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(INT_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeIntArray(val);
-        }
-    }
-
-    /**
-     * @param val Long array.
-     */
-    void doWriteLongArray(@Nullable long[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(LONG_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeLongArray(val);
-        }
-    }
-
-    /**
-     * @param val Float array.
-     */
-    void doWriteFloatArray(@Nullable float[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(FLOAT_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeFloatArray(val);
-        }
-    }
-
-    /**
-     * @param val Double array.
-     */
-    void doWriteDoubleArray(@Nullable double[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(DOUBLE_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeDoubleArray(val);
-        }
-    }
-
-    /**
-     * @param val Char array.
-     */
-    void doWriteCharArray(@Nullable char[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(CHAR_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeCharArray(val);
-        }
-    }
-
-    /**
-     * @param val Boolean array.
-     */
-    void doWriteBooleanArray(@Nullable boolean[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(BOOLEAN_ARR);
-            doWriteInt(val.length);
-
-            wCtx.out.writeBooleanArray(val);
-        }
-    }
-
-    /**
-     * @param val Array of strings.
-     */
-    void doWriteDecimalArray(@Nullable BigDecimal[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(DECIMAL_ARR);
-            doWriteInt(val.length);
-
-            for (BigDecimal str : val)
-                doWriteDecimal(str);
-        }
-    }
-
-    /**
-     * @param val Array of strings.
-     */
-    void doWriteStringArray(@Nullable String[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(STRING_ARR);
-            doWriteInt(val.length);
-
-            for (String str : val)
-                doWriteString(str);
-        }
-    }
-
-    /**
-     * @param val Array of UUIDs.
-     */
-    void doWriteUuidArray(@Nullable UUID[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(UUID_ARR);
-            doWriteInt(val.length);
-
-            for (UUID uuid : val)
-                doWriteUuid(uuid);
-        }
-    }
-
-    /**
-     * @param val Array of dates.
-     */
-    void doWriteDateArray(@Nullable Date[] val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            doWriteByte(DATE_ARR);
-            doWriteInt(val.length);
-
-            for (Date date : val)
-                doWriteDate(date);
-        }
-    }
-
-     /**
-      * @param val Array of timestamps.
-      */
-     void doWriteTimestampArray(@Nullable Timestamp[] val) {
-         if (val == null)
-             doWriteByte(NULL);
-         else {
-             if (tryWriteAsHandle(val))
-                 return;
-
-             doWriteByte(TIMESTAMP_ARR);
-             doWriteInt(val.length);
-
-             for (Timestamp ts : val)
-                 doWriteTimestamp(ts);
-         }
-     }
-
-    /**
-     * @param val Array of objects.
-     * @throws PortableException In case of error.
-     */
-    void doWriteObjectArray(@Nullable Object[] val) throws PortableException {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(val))
-                return;
-
-            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
-
-            doWriteByte(OBJ_ARR);
-
-            if (desc.registered())
-                doWriteInt(desc.typeId());
-            else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
-                doWriteString(val.getClass().getComponentType().getName());
-            }
-
-            doWriteInt(val.length);
-
-            for (Object obj : val)
-                doWriteObject(obj, false);
-        }
-    }
-
-    /**
-     * @param col Collection.
-     * @throws PortableException In case of error.
-     */
-    void doWriteCollection(@Nullable Collection<?> col) throws PortableException {
-        if (col == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(col))
-                return;
-
-            doWriteByte(COL);
-            doWriteInt(col.size());
-            doWriteByte(ctx.collectionType(col.getClass()));
-
-            for (Object obj : col)
-                doWriteObject(obj, false);
-        }
-    }
-
-    /**
-     * @param map Map.
-     * @throws PortableException In case of error.
-     */
-    void doWriteMap(@Nullable Map<?, ?> map) throws PortableException {
-        if (map == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(map))
-                return;
-
-            doWriteByte(MAP);
-            doWriteInt(map.size());
-            doWriteByte(ctx.mapType(map.getClass()));
-
-            for (Map.Entry<?, ?> e : map.entrySet()) {
-                doWriteObject(e.getKey(), false);
-                doWriteObject(e.getValue(), false);
-            }
-        }
-    }
-
-    /**
-     * @param e Map entry.
-     * @throws PortableException In case of error.
-     */
-    void doWriteMapEntry(@Nullable Map.Entry<?, ?> e) throws PortableException {
-        if (e == null)
-            doWriteByte(NULL);
-        else {
-            if (tryWriteAsHandle(e))
-                return;
-
-            doWriteByte(MAP_ENTRY);
-            doWriteObject(e.getKey(), false);
-            doWriteObject(e.getValue(), false);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void doWriteEnum(@Nullable Enum<?> val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass());
-
-            doWriteByte(ENUM);
-
-            if (desc.registered())
-                doWriteInt(desc.typeId());
-            else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
-                doWriteString(val.getClass().getName());
-            }
-
-            doWriteInt(val.ordinal());
-        }
-    }
-
-    /**
-     * @param val Array.
-     */
-    void doWriteEnumArray(@Nullable Object[] val) {
-        assert val == null || val.getClass().getComponentType().isEnum();
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
-            doWriteByte(ENUM_ARR);
-
-            if (desc.registered())
-                doWriteInt(desc.typeId());
-            else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
-                doWriteString(val.getClass().getComponentType().getName());
-            }
-
-            doWriteInt(val.length);
-
-            // TODO: Denis: Redundant data for each element of the array.
-            for (Object o : val)
-                doWriteEnum((Enum<?>)o);
-        }
-    }
-
-    /**
-     * @param val Class.
-     */
-    void doWriteClass(@Nullable Class val) {
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            PortableClassDescriptor desc = ctx.descriptorForClass(val);
-
-            doWriteByte(CLASS);
-
-            if (desc.registered())
-                doWriteInt(desc.typeId());
-            else {
-                doWriteInt(UNREGISTERED_TYPE_ID);
-                doWriteString(val.getClass().getName());
-            }
-        }
-    }
-
-    /**
-     * @param po Portable object.
-     */
-    public void doWritePortableObject(@Nullable PortableObjectImpl po) {
-        if (po == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(PORTABLE_OBJ);
-
-            byte[] poArr = po.array();
-
-            doWriteInt(poArr.length);
-
-            wCtx.out.writeByteArray(poArr);
-
-            doWriteInt(po.start());
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeByteField(@Nullable Byte val) {
-        doWriteInt(val != null ? 2 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(BYTE);
-            doWriteByte(val);
-        }
-    }
-
-    /**
-     * @param val Class.
-     */
-    void writeClassField(@Nullable Class val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteClass(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeShortField(@Nullable Short val) {
-        doWriteInt(val != null ? 3 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(SHORT);
-            doWriteShort(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeIntField(@Nullable Integer val) {
-        doWriteInt(val != null ? 5 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(INT);
-            doWriteInt(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeLongField(@Nullable Long val) {
-        doWriteInt(val != null ? 9 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(LONG);
-            doWriteLong(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeFloatField(@Nullable Float val) {
-        doWriteInt(val != null ? 5 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(FLOAT);
-            doWriteFloat(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDoubleField(@Nullable Double val) {
-        doWriteInt(val != null ? 9 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(DOUBLE);
-            doWriteDouble(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeCharField(@Nullable Character val) {
-        doWriteInt(val != null ? 3 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(CHAR);
-            doWriteChar(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeBooleanField(@Nullable Boolean val) {
-        doWriteInt(val != null ? 2 : 1);
-
-        if (val == null)
-            doWriteByte(NULL);
-        else {
-            doWriteByte(BOOLEAN);
-            doWriteBoolean(val);
-        }
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDecimalField(@Nullable BigDecimal val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteDecimal(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeStringField(@Nullable String val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteString(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeUuidField(@Nullable UUID val) {
-        doWriteInt(val != null ? 17 : 1);
-        doWriteUuid(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDateField(@Nullable Date val) {
-        doWriteInt(val != null ? 9 : 1);
-        doWriteDate(val);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeTimestampField(@Nullable Timestamp val) {
-        doWriteInt(val != null ? 13 : 1);
-        doWriteTimestamp(val);
-    }
-
-    /**
-     * @param obj Object.
-     * @throws PortableException In case of error.
-     */
-    void writeObjectField(@Nullable Object obj) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWriteObject(obj, false);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeByteArrayField(@Nullable byte[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteByteArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeShortArrayField(@Nullable short[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteShortArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeIntArrayField(@Nullable int[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteIntArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeLongArrayField(@Nullable long[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteLongArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeFloatArrayField(@Nullable float[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteFloatArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDoubleArrayField(@Nullable double[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteDoubleArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeCharArrayField(@Nullable char[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteCharArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeBooleanArrayField(@Nullable boolean[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteBooleanArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDecimalArrayField(@Nullable BigDecimal[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteDecimalArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeStringArrayField(@Nullable String[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteStringArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeUuidArrayField(@Nullable UUID[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteUuidArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeDateArrayField(@Nullable Date[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteDateArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeTimestampArrayField(@Nullable Timestamp[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteTimestampArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     * @throws PortableException In case of error.
-     */
-    void writeObjectArrayField(@Nullable Object[] val) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWriteObjectArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param col Collection.
-     * @throws PortableException In case of error.
-     */
-    void writeCollectionField(@Nullable Collection<?> col) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWriteCollection(col);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param map Map.
-     * @throws PortableException In case of error.
-     */
-    void writeMapField(@Nullable Map<?, ?> map) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWriteMap(map);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param e Map entry.
-     * @throws PortableException In case of error.
-     */
-    void writeMapEntryField(@Nullable Map.Entry<?, ?> e) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWriteMapEntry(e);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeEnumField(@Nullable Enum<?> val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteEnum(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param val Value.
-     */
-    void writeEnumArrayField(@Nullable Object[] val) {
-        int lenPos = reserveAndMark(4);
-
-        doWriteEnumArray(val);
-
-        writeDelta(lenPos);
-    }
-
-    /**
-     * @param po Portable object.
-     * @throws PortableException In case of error.
-     */
-    void writePortableObjectField(@Nullable PortableObjectImpl po) throws PortableException {
-        int lenPos = reserveAndMark(4);
-
-        doWritePortableObject(po);
-
-        writeDelta(lenPos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByte(String fieldName, byte val) throws PortableException {
-        writeFieldId(fieldName, BYTE);
-        writeByteField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByte(byte val) throws PortableException {
-        doWriteByte(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShort(String fieldName, short val) throws PortableException {
-        writeFieldId(fieldName, SHORT);
-        writeShortField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShort(short val) throws PortableException {
-        doWriteShort(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeInt(String fieldName, int val) throws PortableException {
-        writeFieldId(fieldName, INT);
-        writeIntField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeInt(int val) throws PortableException {
-        doWriteInt(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLong(String fieldName, long val) throws PortableException {
-        writeFieldId(fieldName, LONG);
-        writeLongField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLong(long val) throws PortableException {
-        doWriteLong(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloat(String fieldName, float val) throws PortableException {
-        writeFieldId(fieldName, FLOAT);
-        writeFloatField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloat(float val) throws PortableException {
-        doWriteFloat(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDouble(String fieldName, double val) throws PortableException {
-        writeFieldId(fieldName, DOUBLE);
-        writeDoubleField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDouble(double val) throws PortableException {
-        doWriteDouble(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChar(String fieldName, char val) throws PortableException {
-        writeFieldId(fieldName, CHAR);
-        writeCharField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChar(char val) throws PortableException {
-        doWriteChar(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBoolean(String fieldName, boolean val) throws PortableException {
-        writeFieldId(fieldName, BOOLEAN);
-        writeBooleanField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBoolean(boolean val) throws PortableException {
-        doWriteBoolean(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws PortableException {
-        writeFieldId(fieldName, DECIMAL);
-        writeDecimalField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimal(@Nullable BigDecimal val) throws PortableException {
-        doWriteDecimal(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeString(String fieldName, @Nullable String val) throws PortableException {
-        writeFieldId(fieldName, STRING);
-        writeStringField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeString(@Nullable String val) throws PortableException {
-        doWriteString(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuid(String fieldName, @Nullable UUID val) throws PortableException {
-        writeFieldId(fieldName, UUID);
-        writeUuidField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuid(@Nullable UUID val) throws PortableException {
-        doWriteUuid(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDate(String fieldName, @Nullable Date val) throws PortableException {
-        writeFieldId(fieldName, DATE);
-        writeDateField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDate(@Nullable Date val) throws PortableException {
-        doWriteDate(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws PortableException {
-        writeFieldId(fieldName, TIMESTAMP);
-        writeTimestampField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestamp(@Nullable Timestamp val) throws PortableException {
-        doWriteTimestamp(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObject(String fieldName, @Nullable Object obj) throws PortableException {
-        writeFieldId(fieldName, OBJ);
-        writeObjectField(obj);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObject(@Nullable Object obj) throws PortableException {
-        doWriteObject(obj, false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObjectDetached(@Nullable Object obj) throws PortableException {
-        doWriteObject(obj, true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws PortableException {
-        writeFieldId(fieldName, BYTE_ARR);
-        writeByteArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByteArray(@Nullable byte[] val) throws PortableException {
-        doWriteByteArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws PortableException {
-        writeFieldId(fieldName, SHORT_ARR);
-        writeShortArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShortArray(@Nullable short[] val) throws PortableException {
-        doWriteShortArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws PortableException {
-        writeFieldId(fieldName, INT_ARR);
-        writeIntArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeIntArray(@Nullable int[] val) throws PortableException {
-        doWriteIntArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws PortableException {
-        writeFieldId(fieldName, LONG_ARR);
-        writeLongArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLongArray(@Nullable long[] val) throws PortableException {
-        doWriteLongArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws PortableException {
-        writeFieldId(fieldName, FLOAT_ARR);
-        writeFloatArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloatArray(@Nullable float[] val) throws PortableException {
-        doWriteFloatArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val)
-        throws PortableException {
-        writeFieldId(fieldName, DOUBLE_ARR);
-        writeDoubleArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDoubleArray(@Nullable double[] val) throws PortableException {
-        doWriteDoubleArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws PortableException {
-        writeFieldId(fieldName, CHAR_ARR);
-        writeCharArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeCharArray(@Nullable char[] val) throws PortableException {
-        doWriteCharArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val)
-        throws PortableException {
-        writeFieldId(fieldName, BOOLEAN_ARR);
-        writeBooleanArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBooleanArray(@Nullable boolean[] val) throws PortableException {
-        doWriteBooleanArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val)
-        throws PortableException {
-        writeFieldId(fieldName, DECIMAL_ARR);
-        writeDecimalArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDecimalArray(@Nullable BigDecimal[] val) throws PortableException {
-        doWriteDecimalArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeStringArray(String fieldName, @Nullable String[] val)
-        throws PortableException {
-        writeFieldId(fieldName, STRING_ARR);
-        writeStringArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeStringArray(@Nullable String[] val) throws PortableException {
-        doWriteStringArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws PortableException {
-        writeFieldId(fieldName, UUID_ARR);
-        writeUuidArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUuidArray(@Nullable UUID[] val) throws PortableException {
-        doWriteUuidArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws PortableException {
-        writeFieldId(fieldName, DATE_ARR);
-        writeDateArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDateArray(@Nullable Date[] val) throws PortableException {
-        doWriteDateArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws PortableException {
-        writeFieldId(fieldName, TIMESTAMP_ARR);
-        writeTimestampArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTimestampArray(@Nullable Timestamp[] val) throws PortableException {
-        doWriteTimestampArray(val);
-    }
-
-     /** {@inheritDoc} */
-    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws PortableException {
-        writeFieldId(fieldName, OBJ_ARR);
-        writeObjectArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeObjectArray(@Nullable Object[] val) throws PortableException {
-        doWriteObjectArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
-        throws PortableException {
-        writeFieldId(fieldName, COL);
-        writeCollectionField(col);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> void writeCollection(@Nullable Collection<T> col) throws PortableException {
-        doWriteCollection(col);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map)
-        throws PortableException {
-        writeFieldId(fieldName, MAP);
-        writeMapField(map);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <K, V> void writeMap(@Nullable Map<K, V> map) throws PortableException {
-        doWriteMap(map);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws PortableException {
-        writeFieldId(fieldName, ENUM);
-        writeEnumField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnum(T val) throws PortableException {
-        doWriteEnum(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws PortableException {
-        writeFieldId(fieldName, ENUM_ARR);
-        writeEnumArrayField(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends Enum<?>> void writeEnumArray(T[] val) throws PortableException {
-        doWriteEnumArray(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableRawWriter rawWriter() {
-        if (allowFields) {
-            wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
-
-            allowFields = false;
-        }
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableOutputStream out() {
-        return wCtx.out;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBytes(String s) throws IOException {
-        int len = s.length();
-
-        writeInt(len);
-
-        for (int i = 0; i < len; i++)
-            writeByte(s.charAt(i));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChars(String s) throws IOException {
-        int len = s.length();
-
-        writeInt(len);
-
-        for (int i = 0; i < len; i++)
-            writeChar(s.charAt(i));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUTF(String s) throws IOException {
-        writeString(s);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByte(int v) throws IOException {
-        doWriteByte((byte)v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShort(int v) throws IOException {
-        doWriteShort((short)v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChar(int v) throws IOException {
-        doWriteChar((char)v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(int b) throws IOException {
-        doWriteByte((byte)b);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush() throws IOException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int reserveInt() {
-        return reserve(LEN_INT);
-    }
-
-     /** {@inheritDoc} */
-    @Override public void writeInt(int pos, int val) throws PortableException {
-        wCtx.out.writeInt(pos, val);
-    }
-
-    /**
-     * @param fieldName Field name.
-     * @throws PortableException If fields are not allowed.
-     */
-    private void writeFieldId(String fieldName, byte fieldType) throws PortableException {
-        A.notNull(fieldName, "fieldName");
-
-        if (!allowFields)
-            throw new PortableException("Individual field can't be written after raw writer is acquired " +
-                "via rawWriter() method. Consider fixing serialization logic for class: " + cls.getName());
-
-        int id = ctx.fieldId(typeId, fieldName);
-
-        if (metaEnabled)
-            metaHashSum = 31 * metaHashSum + (id + fieldType);
-
-        doWriteInt(id);
-    }
-
-     /**
-      * Attempts to write the object as a handle.
-      *
-      * @param obj Object to write.
-      * @return {@code true} if the object has been written as a handle.
-      */
-     boolean tryWriteAsHandle(Object obj) {
-         int handle = handle(obj);
-
-         if (handle >= 0) {
-             doWriteByte(GridPortableMarshaller.HANDLE);
-             doWriteInt(handle);
-
-             return true;
-         }
-
-         return false;
-     }
-
-    /**
-     * Create new writer with same context.
-     * @param typeId type
-     * @return New writer.
-     */
-    public PortableWriterExImpl newWriter(int typeId) {
-        PortableWriterExImpl res = new PortableWriterExImpl(ctx, wCtx);
-
-        res.typeId = typeId;
-
-        return res;
-    }
-
-    /**
-     * @return Portable context.
-     */
-    public PortableContext context() {
-        return ctx;
-    }
-
-    /** */
-    private static class WriterContext {
-        /** */
-        private Map<Object, Integer> handles = new IdentityHashMap<>();
-
-        /** Output stream. */
-        private PortableOutputStream out;
-
-        /**
-         * Constructor.
-         *
-         * @param out Output stream.
-         * @param handles Handles.
-         */
-        private WriterContext(PortableOutputStream out, Map<Object, Integer> handles) {
-            this.out = out;
-            this.handles = handles == null ? new IdentityHashMap<Object, Integer>() : handles;
-        }
-
-        /**
-         * @param obj Object.
-         * @return Handle.
-         */
-        private int handle(Object obj) {
-            assert obj != null;
-
-            Integer h = handles.get(obj);
-
-            if (h != null)
-                return out.position() - h;
-            else {
-                handles.put(obj, out.position());
-
-                return -1;
-            }
-        }
-
-        /**
-         *
-         */
-        private void resetHandles() {
-            handles = new IdentityHashMap<>();
-        }
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
index 1472d56..a97d8e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
@@ -18,9 +18,9 @@
 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.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
 
 /**
  *
@@ -61,7 +61,7 @@ public class PortableBuilderEnum implements PortableBuilderSerializationAware {
                 cls = U.forName(reader.readString(), null);
             }
             catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+                throw new IgniteObjectInvalidClassException("Failed to load the class: " + clsName, e);
             }
 
             this.typeId = reader.portableContext().descriptorForClass(cls).typeId();
@@ -82,7 +82,7 @@ public class PortableBuilderEnum implements PortableBuilderSerializationAware {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         writer.writeByte(GridPortableMarshaller.ENUM);
 
         if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {


[08/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
deleted file mode 100644
index 66b8f76..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableObject.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Wrapper for portable object in portable binary format. Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * <b>NOTE:</b> user does not need to (and should not) implement this interface directly.
- * <p>
- * To work with the portable format directly, user should create a cache projection
- * over {@code PortableObject} class and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use GridPortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
- *
- * builder.setField("fieldA", "A");
- * builder.setField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject.toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- */
-public interface PortableObject extends Serializable, Cloneable {
-    /**
-     * Gets portable object type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId();
-
-    /**
-     * Gets meta data for this portable object.
-     *
-     * @return Meta data.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metaData() throws PortableException;
-
-    /**
-     * Gets field value.
-     *
-     * @param fieldName Field name.
-     * @return Field value.
-     * @throws PortableException In case of any other error.
-     */
-    @Nullable public <F> F field(String fieldName) throws PortableException;
-
-    /**
-     * Checks whether field is set.
-     *
-     * @param fieldName Field name.
-     * @return {@code true} if field is set.
-     */
-    public boolean hasField(String fieldName);
-
-    /**
-     * Gets fully deserialized instance of portable object.
-     *
-     * @return Fully deserialized instance of portable object.
-     * @throws PortableInvalidClassException If class doesn't exist.
-     * @throws PortableException In case of any other error.
-     */
-    @Nullable public <T> T deserialize() throws PortableException;
-
-    /**
-     * Copies this portable object.
-     *
-     * @return Copy of this portable object.
-     */
-    public PortableObject clone() throws CloneNotSupportedException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
deleted file mode 100644
index f2b1cda..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableReader.java
+++ /dev/null
@@ -1,291 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Reader for portable objects used in {@link PortableMarshalAware} implementations.
- * Useful for the cases when user wants a fine-grained control over serialization.
- * <p>
- * Note that Ignite never writes full strings for field or type names. Instead,
- * for performance reasons, Ignite writes integer hash codes for type and field names.
- * It has been tested that hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide, Ignite provides {@link PortableIdMapper} which
- * allows to override the automatically generated hash code IDs for the type and field names.
- */
-public interface PortableReader {
-    /**
-     * @param fieldName Field name.
-     * @return Byte value.
-     * @throws PortableException In case of error.
-     */
-    public byte readByte(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Short value.
-     * @throws PortableException In case of error.
-     */
-    public short readShort(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Integer value.
-     * @throws PortableException In case of error.
-     */
-    public int readInt(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Long value.
-     * @throws PortableException In case of error.
-     */
-    public long readLong(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @throws PortableException In case of error.
-     * @return Float value.
-     */
-    public float readFloat(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Double value.
-     * @throws PortableException In case of error.
-     */
-    public double readDouble(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Char value.
-     * @throws PortableException In case of error.
-     */
-    public char readChar(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Boolean value.
-     * @throws PortableException In case of error.
-     */
-    public boolean readBoolean(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Decimal value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal readDecimal(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return String value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String readString(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return UUID.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID readUuid(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Date.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date readDate(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Timestamp.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Timestamp readTimestamp(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> T readObject(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public byte[] readByteArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Short array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public short[] readShortArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Integer array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public int[] readIntArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Long array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public long[] readLongArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Float array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public float[] readFloatArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Byte array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public double[] readDoubleArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Char array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public char[] readCharArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Boolean array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public boolean[] readBooleanArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Decimal array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return String array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public String[] readStringArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return UUID array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public UUID[] readUuidArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Date array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Date[] readDateArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Timestamp array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Timestamp[] readTimestampArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Object array.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Object[] readObjectArray(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param colCls Collection class.
-     * @return Collection.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
-        throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param mapCls Map class.
-     * @return Map.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
-        throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws PortableException;
-
-    /**
-     * Gets raw reader. Raw reader does not use field name hash codes, therefore,
-     * making the format even more compact. However, if the raw reader is used,
-     * dynamic structure changes to the portable objects are not supported.
-     *
-     * @return Raw reader.
-     */
-    public PortableRawReader rawReader();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
deleted file mode 100644
index 90ee562..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableSerializer.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Interface that allows to implement custom serialization logic for portable objects.
- * Can be used instead of {@link PortableMarshalAware} in case if the class
- * cannot be changed directly.
- * <p>
- * Portable serializer can be configured for all portable objects via
- * {@link PortableMarshaller#getSerializer()} method, or for a specific
- * portable type via {@link PortableTypeConfiguration#getSerializer()} method.
- */
-public interface PortableSerializer {
-    /**
-     * Writes fields to provided writer.
-     *
-     * @param obj Empty object.
-     * @param writer Portable object writer.
-     * @throws PortableException In case of error.
-     */
-    public void writePortable(Object obj, PortableWriter writer) throws PortableException;
-
-    /**
-     * Reads fields from provided reader.
-     *
-     * @param obj Empty object
-     * @param reader Portable object reader.
-     * @throws PortableException In case of error.
-     */
-    public void readPortable(Object obj, PortableReader reader) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
deleted file mode 100644
index 68f0514..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableTypeConfiguration.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import java.sql.Timestamp;
-import java.util.Collection;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Defines configuration properties for a specific portable type. Providing per-type
- * configuration is optional, as it is generally enough, and also optional, to provide global portable
- * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
- * However, this class allows you to change configuration properties for a specific
- * portable type without affecting configuration for other portable types.
- * <p>
- * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
- */
-public class PortableTypeConfiguration {
-    /** Class name. */
-    private String clsName;
-
-    /** ID mapper. */
-    private PortableIdMapper idMapper;
-
-    /** Serializer. */
-    private PortableSerializer serializer;
-
-    /** Meta data enabled flag. */
-    private Boolean metaDataEnabled;
-
-    /** Keep deserialized flag. */
-    private Boolean keepDeserialized;
-
-    /** Affinity key field name. */
-    private String affKeyFieldName;
-
-    /**
-     */
-    public PortableTypeConfiguration() {
-        // No-op.
-    }
-
-    /**
-     * @param clsName Class name.
-     */
-    public PortableTypeConfiguration(String clsName) {
-        this.clsName = clsName;
-    }
-
-    /**
-     * Gets type name.
-     *
-     * @return Type name.
-     */
-    public String getClassName() {
-        return clsName;
-    }
-
-    /**
-     * Sets type name.
-     *
-     * @param clsName Type name.
-     */
-    public void setClassName(String clsName) {
-        this.clsName = clsName;
-    }
-
-    /**
-     * Gets ID mapper.
-     *
-     * @return ID mapper.
-     */
-    public PortableIdMapper getIdMapper() {
-        return idMapper;
-    }
-
-    /**
-     * Sets ID mapper.
-     *
-     * @param idMapper ID mapper.
-     */
-    public void setIdMapper(PortableIdMapper idMapper) {
-        this.idMapper = idMapper;
-    }
-
-    /**
-     * Gets serializer.
-     *
-     * @return Serializer.
-     */
-    public PortableSerializer getSerializer() {
-        return serializer;
-    }
-
-    /**
-     * Sets serializer.
-     *
-     * @param serializer Serializer.
-     */
-    public void setSerializer(PortableSerializer serializer) {
-        this.serializer = serializer;
-    }
-
-    /**
-     * Defines whether meta data is collected for this type. If provided, this value will override
-     * {@link PortableMarshaller#isMetaDataEnabled()} property.
-     *
-     * @return Whether meta data is collected.
-     */
-    public Boolean isMetaDataEnabled() {
-        return metaDataEnabled;
-    }
-
-    /**
-     * @param metaDataEnabled Whether meta data is collected.
-     */
-    public void setMetaDataEnabled(Boolean metaDataEnabled) {
-        this.metaDataEnabled = metaDataEnabled;
-    }
-
-    /**
-     * Defines whether {@link PortableObject} should cache deserialized instance. If provided,
-     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
-     * property.
-     *
-     * @return Whether deserialized value is kept.
-     */
-    public Boolean isKeepDeserialized() {
-        return keepDeserialized;
-    }
-
-    /**
-     * @param keepDeserialized Whether deserialized value is kept.
-     */
-    public void setKeepDeserialized(Boolean keepDeserialized) {
-        this.keepDeserialized = keepDeserialized;
-    }
-
-    /**
-     * Gets affinity key field name.
-     *
-     * @return Affinity key field name.
-     */
-    public String getAffinityKeyFieldName() {
-        return affKeyFieldName;
-    }
-
-    /**
-     * Sets affinity key field name.
-     *
-     * @param affKeyFieldName Affinity key field name.
-     */
-    public void setAffinityKeyFieldName(String affKeyFieldName) {
-        this.affKeyFieldName = affKeyFieldName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableTypeConfiguration.class, this, super.toString());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
deleted file mode 100644
index 99bd5c6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableWriter.java
+++ /dev/null
@@ -1,273 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Writer for portable object used in {@link PortableMarshalAware} implementations.
- * Useful for the cases when user wants a fine-grained control over serialization.
- * <p>
- * Note that Ignite never writes full strings for field or type names. Instead,
- * for performance reasons, Ignite writes integer hash codes for type and field names.
- * It has been tested that hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide, Ignite provides {@link PortableIdMapper} which
- * allows to override the automatically generated hash code IDs for the type and field names.
- */
-public interface PortableWriter {
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByte(String fieldName, byte val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShort(String fieldName, short val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeInt(String fieldName, int val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLong(String fieldName, long val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloat(String fieldName, float val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDouble(String fieldName, double val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeChar(String fieldName, char val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBoolean(String fieldName, boolean val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeString(String fieldName, @Nullable String val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val UUID to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuid(String fieldName, @Nullable UUID val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Date to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDate(String fieldName, @Nullable Date val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Timestamp to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param obj Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObject(String fieldName, @Nullable Object obj) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeByteArray(String fieldName, @Nullable byte[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeShortArray(String fieldName, @Nullable short[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeIntArray(String fieldName, @Nullable int[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeLongArray(String fieldName, @Nullable long[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeFloatArray(String fieldName, @Nullable float[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeCharArray(String fieldName, @Nullable char[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeStringArray(String fieldName, @Nullable String[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeDateArray(String fieldName, @Nullable Date[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param col Collection to write.
-     * @throws PortableException In case of error.
-     */
-    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param map Map to write.
-     * @throws PortableException In case of error.
-     */
-    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws PortableException;
-
-    /**
-     * @param fieldName Field name.
-     * @param val Value to write.
-     * @throws PortableException In case of error.
-     */
-    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws PortableException;
-
-    /**
-     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
-     * making the format even more compact. However, if the raw writer is used,
-     * dynamic structure changes to the portable objects are not supported.
-     *
-     * @return Raw writer.
-     */
-    public PortableRawWriter rawWriter();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/package-info.java b/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
deleted file mode 100644
index 0105b15..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/package-info.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
- * <!-- Package description. -->
- * Contains portable objects API classes.
- */
-package org.apache.ignite.portable;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderAdditionalSelfTest.java
new file mode 100644
index 0000000..d9fbc1c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderAdditionalSelfTest.java
@@ -0,0 +1,1289 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteObjects;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
+import org.apache.ignite.internal.portable.builder.IgniteObjectBuilderImpl;
+import org.apache.ignite.internal.portable.mutabletest.GridIgniteObjectMarshalerAwareTestClass;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.processors.cache.portable.IgniteObjectsImpl;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+
+/**
+ *
+ */
+public class GridIgniteObjectBuilderAdditionalSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheMode(REPLICATED);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
+
+        marsh.setConvertStringToBytes(useUtf8());
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        jcache(0).clear();
+    }
+
+    /**
+     * @return Whether to use UTF8 strings.
+     */
+    protected boolean useUtf8() {
+        return true;
+    }
+
+    /**
+     * @return Portables API.
+     */
+    protected IgniteObjects portables() {
+        return grid(0).portables();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleTypeFieldRead() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        IgniteObjectBuilder mutPo = wrap(exp);
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            Object expVal = field.get(exp);
+            Object actVal = mutPo.getField(field.getName());
+
+            switch (field.getName()) {
+                case "anEnum":
+                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
+                    break;
+
+                case "enumArr": {
+                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
+                    Enum[] expArr = (Enum[])expVal;
+
+                    assertEquals(expArr.length, actArr.length);
+
+                    for (int i = 0; i < actArr.length; i++)
+                        assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
+
+                    break;
+                }
+
+                case "entry":
+                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
+                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
+                    break;
+
+                default:
+                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
+                    break;
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    public void testSimpleTypeFieldSerialize() {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        IgniteObjectBuilderImpl mutPo = wrap(exp);
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        GridTestUtils.deepEquals(exp, res);
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testSimpleTypeFieldOverride() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        IgniteObjectBuilderImpl mutPo = wrap(new TestObjectAllTypes());
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields())
+            mutPo.setField(field.getName(), field.get(exp));
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        GridTestUtils.deepEquals(exp, res);
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testSimpleTypeFieldSetNull() throws Exception {
+        TestObjectAllTypes exp = new TestObjectAllTypes();
+
+        exp.setDefaultData();
+
+        IgniteObjectBuilderImpl mutPo = wrap(exp);
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            if (!field.getType().isPrimitive())
+                mutPo.setField(field.getName(), null);
+        }
+
+        TestObjectAllTypes res = mutPo.build().deserialize();
+
+        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
+            if (!field.getType().isPrimitive())
+                assertNull(field.getName(), field.get(res));
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If any error occurs.
+     */
+    public void testMakeCyclicDependency() throws IgniteCheckedException {
+        TestObjectOuter outer = new TestObjectOuter();
+        outer.inner = new TestObjectInner();
+
+        IgniteObjectBuilderImpl mutOuter = wrap(outer);
+
+        IgniteObjectBuilderImpl mutInner = mutOuter.getField("inner");
+
+        mutInner.setField("outer", mutOuter);
+        mutInner.setField("foo", mutInner);
+
+        TestObjectOuter res = mutOuter.build().deserialize();
+
+        assertEquals(res, res.inner.outer);
+        assertEquals(res.inner, res.inner.foo);
+    }
+
+    /**
+     *
+     */
+    public void testDateArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Date[] arr = mutObj.getField("dateArr");
+        arr[0] = new Date(22222);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
+    }
+
+    /**
+     *
+     */
+    public void testTimestampArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.tsArr = new Timestamp[] {new Timestamp(111222333), new Timestamp(222333444)};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Timestamp[] arr = mutObj.getField("tsArr");
+        arr[0] = new Timestamp(333444555);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Timestamp[] {new Timestamp(333444555), new Timestamp(222333444)}, res.tsArr);
+    }
+
+    /**
+     *
+     */
+    public void testUUIDArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        UUID[] arr = mutObj.getField("uuidArr");
+        arr[0] = new UUID(2, 2);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
+    }
+
+    /**
+     *
+     */
+    public void testDecimalArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        BigDecimal[] arr = mutObj.getField("bdArr");
+        arr[0] = new BigDecimal(2000);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
+            res.bdArr);
+    }
+
+    /**
+     *
+     */
+    public void testBooleanArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.zArr = new boolean[] {false, false, false};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        boolean[] arr = mutObj.getField("zArr");
+        arr[0] = true;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        boolean[] expected = new boolean[] {true, false, false};
+
+        assertEquals(expected.length, res.zArr.length);
+
+        for (int i = 0; i < expected.length; i++)
+            assertEquals(expected[i], res.zArr[i]);
+    }
+
+    /**
+     *
+     */
+    public void testCharArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.cArr = new char[] {'a', 'a', 'a'};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        char[] arr = mutObj.getField("cArr");
+        arr[0] = 'b';
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
+    }
+
+    /**
+     *
+     */
+    public void testDoubleArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.dArr = new double[] {1.0, 1.0, 1.0};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        double[] arr = mutObj.getField("dArr");
+        arr[0] = 2.0;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testFloatArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        float[] arr = mutObj.getField("fArr");
+        arr[0] = 2.0f;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
+    }
+
+    /**
+     *
+     */
+    public void testLongArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.lArr = new long[] {1, 1, 1};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        long[] arr = mutObj.getField("lArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
+    }
+
+    /**
+     *
+     */
+    public void testIntArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.iArr = new int[] {1, 1, 1};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        int[] arr = mutObj.getField("iArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
+    }
+
+    /**
+     *
+     */
+    public void testShortArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.sArr = new short[] {1, 1, 1};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        short[] arr = mutObj.getField("sArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
+    }
+
+    /**
+     *
+     */
+    public void testByteArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.bArr = new byte[] {1, 1, 1};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        byte[] arr = mutObj.getField("bArr");
+        arr[0] = 2;
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
+    }
+
+    /**
+     *
+     */
+    public void testStringArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.strArr = new String[] {"a", "a", "a"};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        String[] arr = mutObj.getField("strArr");
+        arr[0] = "b";
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
+    }
+
+    /**
+     *
+     */
+    public void testModifyObjectArray() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = new Object[] {"a"};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Object[] arr = mutObj.getField("foo");
+
+        Assert.assertArrayEquals(new Object[] {"a"}, arr);
+
+        arr[0] = "b";
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testOverrideObjectArrayField() {
+        IgniteObjectBuilderImpl mutObj = wrap(new TestObjectContainer());
+
+        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
+
+        mutObj.setField("foo", createdArr.clone());
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        createdArr[0] = res;
+
+        assertTrue(Objects.deepEquals(createdArr, res.foo));
+    }
+
+    /**
+     *
+     */
+    public void testDeepArray() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = new Object[] {new Object[] {"a", obj}};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
+
+        assertEquals("a", arr[0]);
+        assertSame(mutObj, arr[1]);
+
+        arr[0] = mutObj;
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        arr = (Object[])((Object[])res.foo)[0];
+
+        assertSame(arr[0], res);
+        assertSame(arr[0], arr[1]);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList(obj, "a");
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = mutObj.getField("foo");
+
+        assert list.equals(Lists.newArrayList(mutObj, "a"));
+    }
+
+    /**
+     *
+     */
+    public void testArrayListOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
+
+        mutObj.setField("foo", list);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        list.set(0, res);
+
+        assertNotSame(list, res.foo);
+        assertEquals(list, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList("a", "b", "c");
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.add("!"); // "a", "b", "c", "!"
+        list.add(0, "_"); // "_", "a", "b", "c", "!"
+
+        String s = list.remove(1); // "_", "b", "c", "!"
+        assertEquals("a", s);
+
+        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+        assertEquals(1, list.indexOf("b"));
+        assertEquals(1, list.lastIndexOf("b"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof ArrayList);
+        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListClear() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList("a", "b", "c");
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.clear();
+
+        assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
+    }
+
+    /**
+     *
+     */
+    public void testArrayListWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
+
+        obj.foo = src;
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        TestObjectContainer deserialized = mutObj.build().deserialize();
+
+        List<Object> res = (List<Object>)deserialized.foo;
+
+        src.set(0, deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = mutObj.getField("foo");
+
+        assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
+
+        mutObj.setField("foo", list);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        list.set(0, res);
+
+        assertNotSame(list, res.foo);
+        assertEquals(list, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        List<String> list = mutObj.getField("foo");
+
+        list.add("!"); // "a", "b", "c", "!"
+        list.add(0, "_"); // "_", "a", "b", "c", "!"
+
+        String s = list.remove(1); // "_", "b", "c", "!"
+        assertEquals("a", s);
+
+        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
+        assertEquals(1, list.indexOf("b"));
+        assertEquals(1, list.lastIndexOf("b"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof LinkedList);
+        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testLinkedListWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
+
+        obj.foo = src;
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        TestObjectContainer deserialized = mutObj.build().deserialize();
+
+        List<Object> res = (List<Object>)deserialized.foo;
+
+        src.set(0, deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Sets.newHashSet(obj, "a");
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<Object> set = mutObj.getField("foo");
+
+        assert set.equals(Sets.newHashSet(mutObj, "a"));
+    }
+
+    /**
+     *
+     */
+    public void testHashSetOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
+
+        mutObj.setField("foo", c);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        c.remove(mutObj);
+        c.add(res);
+
+        assertNotSame(c, res.foo);
+        assertEquals(c, res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Sets.newHashSet("a", "b", "c");
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Set<String> set = mutObj.getField("foo");
+
+        set.remove("b");
+        set.add("!");
+
+        assertEquals(Sets.newHashSet("a", "!", "c"), set);
+        assertTrue(set.contains("a"));
+        assertTrue(set.contains("!"));
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertTrue(res.foo instanceof HashSet);
+        assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testHashSetWriteUnmodifiable() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
+
+        obj.foo = src;
+
+        TestObjectContainer deserialized = wrap(obj).build().deserialize();
+
+        Set<Object> res = (Set<Object>)deserialized.foo;
+
+        src.remove(obj);
+        src.add(deserialized);
+
+        assertEquals(src, res);
+    }
+
+    /**
+     *
+     */
+    public void testMapRead() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = mutObj.getField("foo");
+
+        assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
+    }
+
+    /**
+     *
+     */
+    public void testMapOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
+
+        mutObj.setField("foo", map);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        Map<Object, Object> map = mutObj.getField("foo");
+
+        map.put(3, mutObj);
+        Object rmv = map.remove(1);
+
+        assertEquals("a", rmv);
+
+        TestObjectContainer res = mutObj.build().deserialize();
+
+        assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testEnumArrayModification() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+
+        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
+
+        IgniteObjectBuilderImpl mutObj = wrap(obj);
+
+        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
+        arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
+
+        TestObjectAllTypes res = mutObj.build().deserialize();
+
+        Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
+    }
+
+    /**
+     *
+     */
+    public void testEditObjectWithRawData() {
+        GridIgniteObjectMarshalerAwareTestClass obj = new GridIgniteObjectMarshalerAwareTestClass();
+
+        obj.s = "a";
+        obj.sRaw = "aa";
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        mutableObj.setField("s", "z");
+
+        GridIgniteObjectMarshalerAwareTestClass res = mutableObj.build().deserialize();
+        assertEquals("z", res.s);
+        assertEquals("aa", res.sRaw);
+    }
+
+    /**
+     *
+     */
+    public void testHashCode() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(obj.hashCode(), mutableObj.build().hashCode());
+
+        mutableObj.hashCode(25);
+
+        assertEquals(25, mutableObj.build().hashCode());
+    }
+
+    /**
+     *
+     */
+    public void testCollectionsInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = Lists.newArrayList(
+            Lists.newArrayList(1, 2),
+            Lists.newLinkedList(Arrays.asList(1, 2)),
+            Sets.newHashSet("a", "b"),
+            Sets.newLinkedHashSet(Arrays.asList("a", "b")),
+            Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
+
+        TestObjectContainer deserialized = wrap(obj).build().deserialize();
+
+        assertEquals(obj.foo, deserialized.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapEntryModification() {
+        TestObjectContainer obj = new TestObjectContainer();
+        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
+
+        assertEquals(1, entry.getKey());
+        assertEquals("a", entry.getValue());
+
+        entry.setValue("b");
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMapEntryOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(new GridMapEntry<>(1, "a"), res.foo);
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChangingDoublePut() {
+        IgniteObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+
+        mutableObj.setField("xx567", "a");
+        mutableObj.setField("xx567", "b");
+
+        mutableObj.build();
+
+        IgniteObjectMetadata metadata = portables().metadata(TestObjectContainer.class);
+
+        assertEquals("String", metadata.fieldTypeName("xx567"));
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChangingDoublePut2() {
+        IgniteObjectBuilderImpl mutableObj = wrap(new TestObjectContainer());
+
+        mutableObj.setField("xx567", "a");
+        mutableObj.setField("xx567", "b");
+
+        mutableObj.build();
+
+        IgniteObjectMetadata metadata = portables().metadata(TestObjectContainer.class);
+
+        assertEquals("String", metadata.fieldTypeName("xx567"));
+    }
+
+    /**
+     *
+     */
+    public void testMetadataChanging() {
+        TestObjectContainer c = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(c);
+
+        mutableObj.setField("intField", 1);
+        mutableObj.setField("intArrField", new int[] {1});
+        mutableObj.setField("arrField", new String[] {"1"});
+        mutableObj.setField("strField", "1");
+        mutableObj.setField("colField", Lists.newArrayList("1"));
+        mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
+        mutableObj.setField("enumField", TestObjectEnum.A);
+        mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
+
+        mutableObj.build();
+
+        IgniteObjectMetadata metadata = portables().metadata(c.getClass());
+
+        assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
+            "colField", "mapField", "enumField", "enumArrField")));
+
+        assertEquals("int", metadata.fieldTypeName("intField"));
+        assertEquals("int[]", metadata.fieldTypeName("intArrField"));
+        assertEquals("String[]", metadata.fieldTypeName("arrField"));
+        assertEquals("String", metadata.fieldTypeName("strField"));
+        assertEquals("Collection", metadata.fieldTypeName("colField"));
+        assertEquals("Map", metadata.fieldTypeName("mapField"));
+        assertEquals("Enum", metadata.fieldTypeName("enumField"));
+        assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
+    }
+
+    /**
+     *
+     */
+    public void testDateInObjectField() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = new Date();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Date.class, mutableObj.getField("foo").getClass());
+    }
+
+    /**
+     *
+     */
+    public void testTimestampInObjectField() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = new Timestamp(100020003);
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
+    }
+
+    /**
+     *
+     */
+    public void testDateInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newArrayList(new Date());
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Date.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+    }
+
+    /**
+     *
+     */
+    public void testTimestampInCollection() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        obj.foo = Lists.newArrayList(new Timestamp(100020003));
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+    public void testDateArrayOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        Date[] arr = { new Date() };
+
+        mutableObj.setField("foo", arr);
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(Date[].class, res.foo.getClass());
+        assertTrue(Objects.deepEquals(arr, res.foo));
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
+    public void testTimestampArrayOverride() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl mutableObj = wrap(obj);
+
+        Timestamp[] arr = { new Timestamp(100020003) };
+
+        mutableObj.setField("foo", arr);
+
+        TestObjectContainer res = mutableObj.build().deserialize();
+
+        assertEquals(Timestamp[].class, res.foo.getClass());
+        assertTrue(Objects.deepEquals(arr, res.foo));
+    }
+
+    /**
+     *
+     */
+    public void testChangeMap() {
+        AddressBook addrBook = new AddressBook();
+
+        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
+        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
+        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
+        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
+
+        IgniteObjectBuilderImpl mutableObj = wrap(addrBook);
+
+        Map<String, List<IgniteObjectBuilderImpl>> map = mutableObj.getField("companyByStreet");
+
+        List<IgniteObjectBuilderImpl> list = map.get("Torzhkovskya");
+
+        IgniteObjectBuilderImpl company = list.get(0);
+
+        assert "Google inc".equals(company.<String>getField("name"));
+
+        list.remove(0);
+
+        AddressBook res = mutableObj.build().deserialize();
+
+        assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
+
+        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
+
+        assertEquals(2, torzhkovskyaCompanies.size());
+        assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
+    }
+
+    /**
+     *
+     */
+    public void testSavingObjectWithNotZeroStart() {
+        TestObjectOuter out = new TestObjectOuter();
+        TestObjectInner inner = new TestObjectInner();
+
+        out.inner = inner;
+        inner.outer = out;
+
+        IgniteObjectBuilderImpl builder = wrap(out);
+
+        IgniteObjectBuilderImpl innerBuilder = builder.getField("inner");
+
+        TestObjectInner res = innerBuilder.build().deserialize();
+
+        assertSame(res, res.outer.inner);
+    }
+
+    /**
+     *
+     */
+    public void testPortableObjectField() {
+        TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
+
+        IgniteObjectBuilderImpl wrapper = wrap(container);
+
+        assertTrue(wrapper.getField("foo") instanceof IgniteObject);
+
+        TestObjectContainer deserialized = wrapper.build().deserialize();
+        assertTrue(deserialized.foo instanceof IgniteObject);
+    }
+
+    /**
+     *
+     */
+    public void testAssignPortableObject() {
+        TestObjectContainer container = new TestObjectContainer();
+
+        IgniteObjectBuilderImpl wrapper = wrap(container);
+
+        wrapper.setField("foo", toPortable(new TestObjectArrayList()));
+
+        TestObjectContainer deserialized = wrapper.build().deserialize();
+        assertTrue(deserialized.foo instanceof TestObjectArrayList);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromNewObject() {
+        IgniteObjectBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
+
+        wrapper.setField("str", "a");
+
+        wrapper.removeField("str");
+
+        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObject() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+
+        IgniteObjectBuilderImpl wrapper = wrap(toPortable(obj));
+
+        wrapper.removeField("str");
+
+        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testCyclicArrays() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        Object[] arr1 = new Object[1];
+        Object[] arr2 = new Object[] {arr1};
+
+        arr1[0] = arr2;
+
+        obj.foo = arr1;
+
+        TestObjectContainer res = toPortable(obj).deserialize();
+
+        Object[] resArr = (Object[])res.foo;
+
+        assertSame(((Object[])resArr[0])[0], resArr);
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    public void testCyclicArrayList() {
+        TestObjectContainer obj = new TestObjectContainer();
+
+        List<Object> arr1 = new ArrayList<>();
+        List<Object> arr2 = new ArrayList<>();
+
+        arr1.add(arr2);
+        arr2.add(arr1);
+
+        obj.foo = arr1;
+
+        TestObjectContainer res = toPortable(obj).deserialize();
+
+        List<?> resArr = (List<?>)res.foo;
+
+        assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
+    }
+
+    /**
+     * @param obj Object.
+     * @return Object in portable format.
+     */
+    private IgniteObject toPortable(Object obj) {
+        return portables().toPortable(obj);
+    }
+
+    /**
+     * @param obj Object.
+     * @return GridMutablePortableObject.
+     */
+    private IgniteObjectBuilderImpl wrap(Object obj) {
+        return IgniteObjectBuilderImpl.wrap(toPortable(obj));
+    }
+
+    /**
+     * @param aCls Class.
+     * @return Wrapper.
+     */
+    private IgniteObjectBuilderImpl newWrapper(Class<?> aCls) {
+        CacheObjectPortableProcessorImpl processor = (CacheObjectPortableProcessorImpl)(
+            (IgniteObjectsImpl)portables()).processor();
+
+        return new IgniteObjectBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
+            aCls.getSimpleName());
+    }
+}
\ No newline at end of file


[13/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
deleted file mode 100644
index 5d1460c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ /dev/null
@@ -1,3230 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
-import org.apache.ignite.internal.portable.streams.PortableInputStream;
-import org.apache.ignite.internal.util.GridEnumCache;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableReader;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-import java.io.ByteArrayInputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.lang.reflect.Array;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.Properties;
-import java.util.TreeMap;
-import java.util.TreeSet;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentSkipListSet;
-
-import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ARR_LIST;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_HASH_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_SKIP_LIST_SET;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HANDLE;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_SET;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_HASH_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_HASH_SET;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_LIST;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJECT_TYPE_ID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROPERTIES_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_MAP;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_SET;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_COL;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_SET;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR;
-
-/**
- * Portable reader implementation.
- */
-@SuppressWarnings("unchecked")
-public class PortableReaderExImpl implements PortableReader, PortableRawReaderEx, ObjectInput {
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final PortableInputStream in;
-
-    /** */
-    private final int start;
-
-    /** */
-    private final PortableReaderContext rCtx;
-
-    /** */
-    private final ClassLoader ldr;
-
-    /** */
-    private int off;
-
-    /** */
-    private int rawOff;
-
-    /** */
-    private int len;
-
-    /** */
-    private PortableClassDescriptor desc;
-
-    /** */
-    private int hdrLen;
-
-    /** */
-    private int clsNameLen;
-
-    /** */
-    private Integer typeId;
-
-    /**
-     * @param ctx Context.
-     * @param arr Array.
-     * @param start Start.
-     * @param ldr Class loader.
-     */
-    public PortableReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
-        this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext());
-    }
-
-    /**
-     * @param ctx Context.
-     * @param in Input stream.
-     * @param start Start.
-     */
-    PortableReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
-        this(ctx, in, start, ldr, new PortableReaderContext());
-    }
-
-    /**
-     * @param ctx Context.
-     * @param in Input stream.
-     * @param start Start.
-     * @param rCtx Context.
-     */
-    PortableReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
-        PortableReaderContext rCtx) {
-        this.ctx = ctx;
-        this.in = in;
-        this.start = start;
-        this.ldr = ldr;
-        this.rCtx = rCtx;
-
-        off = start;
-        rawOff = start;
-    }
-
-    /**
-     * Preloads typeId from the input array.
-     */
-    private void readObjectTypeId(boolean skipObjByte) {
-        int pos = rawOff;
-
-        if (!skipObjByte)
-            // skip obj type byte
-            rawOff++;
-
-        // Validate protocol version.
-        PortableUtils.checkProtocolVersion(doReadByte(true));
-
-        // skip user flag
-        rawOff += 1;
-
-        typeId = doReadInt(true);
-
-        if (typeId == UNREGISTERED_TYPE_ID) {
-            // skip hash code, length and raw offset
-            rawOff += 12;
-
-            int off = rawOff;
-
-            Class cls = doReadClass(true, typeId);
-
-            // registers class by typeId, at least locally if the cache is not ready yet.
-            PortableClassDescriptor desc = ctx.descriptorForClass(cls);
-
-            typeId = desc.typeId();
-
-            clsNameLen = rawOff - off;
-
-            hdrLen = CLS_NAME_POS + clsNameLen;
-        }
-        else
-            hdrLen = DFLT_HDR_LEN;
-
-        in.position(rawOff = pos);
-    }
-
-    /**
-     * @return Descriptor.
-     */
-    PortableClassDescriptor descriptor() {
-        return desc;
-    }
-
-    /**
-     * @return Unmarshalled value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Object unmarshal() throws PortableException {
-        return unmarshal(true);
-    }
-
-    /**
-     * @param fieldName Field name.
-     * @return Unmarshalled value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Object unmarshal(String fieldName) throws PortableException {
-        off = fieldOffset(fieldId(fieldName));
-
-        return off >= 0 ? unmarshal(false) : null;
-    }
-
-    /**
-     * @param offset Offset in the array.
-     * @return Unmarshalled value.
-     * @throws PortableException In case of error.
-     */
-    public Object unmarshal(int offset) throws PortableException {
-        off = offset;
-
-        return off >= 0 ? unmarshal(false) : null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Byte readByte(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != BYTE)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadByte(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Short readShort(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != SHORT)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadShort(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Integer readInt(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != INT)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadInt(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Long readLong(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != LONG)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadLong(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Float readFloat(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != FLOAT)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadFloat(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Double readDouble(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != DOUBLE)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDouble(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Character readChar(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != CHAR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadChar(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Boolean readBoolean(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != BOOLEAN)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadBoolean(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable BigDecimal readDecimal(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != DECIMAL)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDecimal(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable String readString(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != STRING)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadString(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable UUID readUuid(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != UUID)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadUuid(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Date readDate(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != DATE)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDate(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Timestamp readTimestamp(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != TIMESTAMP)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadTimestamp(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Object readObject(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        return off >= 0 ? doReadObject(false) : null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable byte[] readByteArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != BYTE_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadByteArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable short[] readShortArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != SHORT_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadShortArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable int[] readIntArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != INT_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadIntArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable long[] readLongArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != LONG_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadLongArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable float[] readFloatArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != FLOAT_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadFloatArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable double[] readDoubleArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != DOUBLE_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDoubleArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable char[] readCharArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != CHAR_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadCharArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable boolean[] readBooleanArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != BOOLEAN_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadBooleanArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable BigDecimal[] readDecimalArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != DECIMAL_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDecimalArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable String[] readStringArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != STRING_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadStringArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable UUID[] readUuidArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != UUID_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadUuidArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Date[] readDateArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != DATE_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadDateArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Timestamp[] readTimestampArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != TIMESTAMP_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadTimestampArray(false);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Object[] readObjectArray(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != OBJ_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadObjectArray(false, true);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @param cls Collection class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable <T> Collection<T> readCollection(int fieldId, @Nullable Class<? extends Collection> cls)
-        throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != COL)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return (Collection<T>)doReadCollection(false, true, cls);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @param cls Map class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Map<?, ?> readMap(int fieldId, @Nullable Class<? extends Map> cls)
-        throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != MAP)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadMap(false, true, cls);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Value.
-     * @throws PortableException On case of error.
-     */
-    @Nullable Map.Entry<?, ?> readMapEntry(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag == HANDLE)
-                return readHandleField();
-
-            if (flag != MAP_ENTRY)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return doReadMapEntry(false, true);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Portable object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable PortableObject readPortableObject(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != PORTABLE_OBJ)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            return new PortableObjectImpl(ctx, doReadByteArray(false), doReadInt(false));
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @param cls Class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Enum<?> readEnum(int fieldId, @Nullable Class<?> cls) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != ENUM)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            // Revisit: why have we started writing Class for enums in their serialized form?
-            if (cls == null)
-                cls = doReadClass(false);
-            else
-                doReadClass(false);
-
-            Object[] vals = GridEnumCache.get(cls);
-
-            return (Enum<?>)vals[doReadInt(false)];
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @param cls Class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Object[] readEnumArray(int fieldId, @Nullable Class<?> cls) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != ENUM_ARR)
-                throw new PortableException("Invalid flag value: " + flag);
-
-            // Revisit: why have we started writing Class for enums in their serialized form?
-            if (cls == null)
-                cls = doReadClass(false);
-            else
-                doReadClass(false);
-
-            return doReadEnumArray(false, cls);
-        }
-        else
-            return null;
-    }
-
-    /**
-     * @param fieldId Field ID.
-     * @return Field class.
-     * @throws PortableException In case of error.
-     */
-    @Nullable Class<?> readClass(int fieldId) throws PortableException {
-        off = fieldOffset(fieldId);
-
-        if (off >= 0) {
-            byte flag = doReadByte(false);
-
-            if (flag == NULL)
-                return null;
-
-            if (flag != CLASS)
-                throw new PortableException("Invalid flag type: [flag=" + flag + ']');
-
-            return doReadClass(false);
-        }
-
-        return null;
-    }
-
-    /**
-     * @param obj Object.
-     */
-    void setHandler(Object obj) {
-        rCtx.setObjectHandler(start, obj);
-    }
-
-    /**
-     * @param obj Object.
-     * @param pos Position.
-     */
-    void setHandler(Object obj, int pos) {
-        rCtx.setObjectHandler(pos, obj);
-    }
-
-    /**
-     * Recreating field value from a handle.
-     *
-     * @param <T> Field type.
-     * @return Field.
-     */
-    private <T> T readHandleField() {
-        int handle = (off - 1) - doReadInt(false);
-
-        Object obj = rCtx.getObjectByHandle(handle);
-
-        if (obj == null) {
-            off = handle;
-
-            obj = doReadObject(false);
-        }
-
-        return (T)obj;
-    }
-    /** {@inheritDoc} */
-    @Override public byte readByte(String fieldName) throws PortableException {
-        Byte val = readByte(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte readByte() throws PortableException {
-        return doReadByte(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public short readShort(String fieldName) throws PortableException {
-        Short val = readShort(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public short readShort() throws PortableException {
-        return doReadShort(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readInt(String fieldName) throws PortableException {
-        Integer val = readInt(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readInt() throws PortableException {
-        return doReadInt(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long readLong(String fieldName) throws PortableException {
-        Long val = readLong(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long readLong() throws PortableException {
-        return doReadLong(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public float readFloat(String fieldName) throws PortableException {
-        Float val = readFloat(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float readFloat() throws PortableException {
-        return doReadFloat(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public double readDouble(String fieldName) throws PortableException {
-        Double val = readDouble(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public double readDouble() throws PortableException {
-        return doReadDouble(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public char readChar(String fieldName) throws PortableException {
-        Character val = readChar(fieldId(fieldName));
-
-        return val != null ? val : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public char readChar() throws PortableException {
-        return doReadChar(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readBoolean(String fieldName) throws PortableException {
-        Boolean val = readBoolean(fieldId(fieldName));
-
-        return val != null ? val : false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readBoolean() throws PortableException {
-        return doReadBoolean(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public BigDecimal readDecimal(String fieldName) throws PortableException {
-        return readDecimal(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public BigDecimal readDecimal() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != DECIMAL)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadDecimal(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String readString(String fieldName) throws PortableException {
-        return readString(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String readString() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != STRING)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadString(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID readUuid(String fieldName) throws PortableException {
-        return readUuid(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID readUuid() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != UUID)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadUuid(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Date readDate(String fieldName) throws PortableException {
-        return readDate(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Date readDate() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != DATE)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadDate(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Timestamp readTimestamp(String fieldName) throws PortableException {
-        return readTimestamp(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Timestamp readTimestamp() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != TIMESTAMP)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadTimestamp(true);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T readObject(String fieldName) throws PortableException {
-        return (T)readObject(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object readObject() throws PortableException {
-        return doReadObject(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Object readObjectDetached() throws PortableException {
-        return unmarshal(true, true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public byte[] readByteArray(String fieldName) throws PortableException {
-        return readByteArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public byte[] readByteArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != BYTE_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadByteArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public short[] readShortArray(String fieldName) throws PortableException {
-        return readShortArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public short[] readShortArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != SHORT_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadShortArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public int[] readIntArray(String fieldName) throws PortableException {
-        return readIntArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public int[] readIntArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != INT_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadIntArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public long[] readLongArray(String fieldName) throws PortableException {
-        return readLongArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public long[] readLongArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != LONG_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadLongArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public float[] readFloatArray(String fieldName) throws PortableException {
-        return readFloatArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public float[] readFloatArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != FLOAT_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadFloatArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public double[] readDoubleArray(String fieldName) throws PortableException {
-        return readDoubleArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public double[] readDoubleArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != DOUBLE_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadDoubleArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public char[] readCharArray(String fieldName) throws PortableException {
-        return readCharArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public char[] readCharArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != CHAR_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadCharArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public boolean[] readBooleanArray(String fieldName) throws PortableException {
-        return readBooleanArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public boolean[] readBooleanArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != BOOLEAN_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadBooleanArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws PortableException {
-        return readDecimalArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public BigDecimal[] readDecimalArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != DECIMAL_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadDecimalArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String[] readStringArray(String fieldName) throws PortableException {
-        return readStringArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String[] readStringArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != STRING_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadStringArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID[] readUuidArray(String fieldName) throws PortableException {
-        return readUuidArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID[] readUuidArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != UUID_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadUuidArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Date[] readDateArray(String fieldName) throws PortableException {
-        return readDateArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Timestamp[] readTimestampArray(String fieldName) throws PortableException {
-        return readTimestampArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Date[] readDateArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != DATE_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadDateArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Timestamp[] readTimestampArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != TIMESTAMP_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadTimestampArray(true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Object[] readObjectArray(String fieldName) throws PortableException {
-        return readObjectArray(fieldId(fieldName));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Object[] readObjectArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != OBJ_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return doReadObjectArray(true, true);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection(String fieldName) throws PortableException {
-        return readCollection(fieldId(fieldName), null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != COL)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return (Collection<T>)doReadCollection(true, true, null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection(String fieldName,
-        Class<? extends Collection<T>> colCls) throws PortableException {
-        return readCollection(fieldId(fieldName), colCls);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
-        throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != COL)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return (Collection<T>)doReadCollection(true, true, colCls);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName) throws PortableException {
-        return (Map<K, V>)readMap(fieldId(fieldName), null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != MAP)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return (Map<K, V>)doReadMap(true, true, null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
-        throws PortableException {
-        return (Map<K, V>)readMap(fieldId(fieldName), mapCls);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls)
-        throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != MAP)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        return (Map<K, V>)doReadMap(true, true, mapCls);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T extends Enum<?>> T readEnum(String fieldName)
-        throws PortableException {
-        return (T)readEnum(fieldId(fieldName), null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T extends Enum<?>> T readEnum() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != ENUM)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        Class cls = doReadClass(true);
-
-        return (T)doReadEnum(true, cls);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray(String fieldName)
-        throws PortableException {
-        return (T[])readEnumArray(fieldId(fieldName), null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray() throws PortableException {
-        byte flag = doReadByte(true);
-
-        if (flag == NULL)
-            return null;
-
-        if (flag != ENUM_ARR)
-            throw new PortableException("Invalid flag value: " + flag);
-
-        Class cls = doReadClass(true);
-
-        return (T[])doReadEnumArray(true, cls);
-    }
-
-    /**
-     * @param fieldName Field name.
-     * @return {@code true} if field is set.
-     */
-    public boolean hasField(String fieldName) {
-        return fieldOffset(fieldId(fieldName)) != -1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableRawReader rawReader() {
-        return this;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Unmarshalled value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable private Object unmarshal(boolean raw) throws PortableException {
-        return unmarshal(raw, false);
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Unmarshalled value.
-     * @throws PortableException In case of error.
-     */
-    @Nullable private Object unmarshal(boolean raw, boolean detach) throws PortableException {
-        int start = raw ? rawOff : off;
-
-        byte flag = doReadByte(raw);
-
-        switch (flag) {
-            case NULL:
-                return null;
-
-            case HANDLE:
-                int handle = start - doReadInt(raw);
-
-                PortableObject handledPo = rCtx.getPortableByHandle(handle);
-
-                if (handledPo != null)
-                    return handledPo;
-
-                off = handle;
-
-                return unmarshal(false);
-
-            case OBJ:
-                PortableUtils.checkProtocolVersion(doReadByte(raw));
-
-                PortableObjectEx po;
-
-                if (detach) {
-                    in.position(start + GridPortableMarshaller.TOTAL_LEN_POS);
-
-                    int len = in.readInt();
-
-                    in.position(start);
-
-                    po = new PortableObjectImpl(ctx, in.readByteArray(len), 0);
-                }
-                else
-                    po = in.offheapPointer() > 0
-                        ? new PortableObjectOffheapImpl(ctx, in.offheapPointer(), start,
-                        in.remaining() + in.position())
-                        : new PortableObjectImpl(ctx, in.array(), start);
-
-                rCtx.setPortableHandler(start, po);
-
-                if (raw)
-                    rawOff = start + po.length();
-                else
-                    off = start + po.length();
-
-                return po;
-
-            case BYTE:
-                return doReadByte(raw);
-
-            case SHORT:
-                return doReadShort(raw);
-
-            case INT:
-                return doReadInt(raw);
-
-            case LONG:
-                return doReadLong(raw);
-
-            case FLOAT:
-                return doReadFloat(raw);
-
-            case DOUBLE:
-                return doReadDouble(raw);
-
-            case CHAR:
-                return doReadChar(raw);
-
-            case BOOLEAN:
-                return doReadBoolean(raw);
-
-            case DECIMAL:
-                return doReadDecimal(raw);
-
-            case STRING:
-                return doReadString(raw);
-
-            case UUID:
-                return doReadUuid(raw);
-
-            case DATE:
-                return doReadDate(raw);
-
-            case TIMESTAMP:
-                return doReadTimestamp(raw);
-
-            case BYTE_ARR:
-                return doReadByteArray(raw);
-
-            case SHORT_ARR:
-                return doReadShortArray(raw);
-
-            case INT_ARR:
-                return doReadIntArray(raw);
-
-            case LONG_ARR:
-                return doReadLongArray(raw);
-
-            case FLOAT_ARR:
-                return doReadFloatArray(raw);
-
-            case DOUBLE_ARR:
-                return doReadDoubleArray(raw);
-
-            case CHAR_ARR:
-                return doReadCharArray(raw);
-
-            case BOOLEAN_ARR:
-                return doReadBooleanArray(raw);
-
-            case DECIMAL_ARR:
-                return doReadDecimalArray(raw);
-
-            case STRING_ARR:
-                return doReadStringArray(raw);
-
-            case UUID_ARR:
-                return doReadUuidArray(raw);
-
-            case DATE_ARR:
-                return doReadDateArray(raw);
-
-            case TIMESTAMP_ARR:
-                return doReadTimestampArray(raw);
-
-            case OBJ_ARR:
-                return doReadObjectArray(raw, false);
-
-            case COL:
-                return doReadCollection(raw, false, null);
-
-            case MAP:
-                return doReadMap(raw, false, null);
-
-            case MAP_ENTRY:
-                return doReadMapEntry(raw, false);
-
-            case PORTABLE_OBJ:
-                return doReadPortableObject(raw);
-
-            case ENUM:
-                return doReadEnum(raw, doReadClass(raw));
-
-            case ENUM_ARR:
-                return doReadEnumArray(raw, doReadClass(raw));
-
-            case CLASS:
-                return doReadInt(raw);
-
-            case OPTM_MARSH:
-                int len = doReadInt(true);
-
-                ByteArrayInputStream input = new ByteArrayInputStream(in.array(), in.position(), len);
-
-                Object obj;
-
-                try {
-                    obj = ctx.optimizedMarsh().unmarshal(input, null);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new PortableException("Failed to unmarshal object with optmMarsh marshaller", e);
-                }
-
-                if (raw)
-                    rawOff += len;
-                else
-                    off += len;
-
-                return obj;
-
-            default:
-                throw new PortableException("Invalid flag value: " + flag);
-        }
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private byte doReadByte(boolean raw) {
-        in.position(raw ? rawOff++ : off++);
-
-        return in.readByte();
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private short doReadShort(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        short val = in.readShort();
-
-        if (raw)
-            rawOff += 2;
-        else
-            off += 2;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private int doReadInt(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        int val = in.readInt();
-
-        if (raw)
-            rawOff += 4;
-        else
-            off += 4;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private long doReadLong(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        long val = in.readLong();
-
-        if (raw)
-            rawOff += 8;
-        else
-            off += 8;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private float doReadFloat(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        float val = in.readFloat();
-
-        if (raw)
-            rawOff += 4;
-        else
-            off += 4;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private double doReadDouble(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        double val = in.readDouble();
-
-        if (raw)
-            rawOff += 8;
-        else
-            off += 8;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private char doReadChar(boolean raw) {
-        in.position(raw ? rawOff : off);
-
-        char val = in.readChar();
-
-        if (raw)
-            rawOff += 2;
-        else
-            off += 2;
-
-        return val;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private boolean doReadBoolean(boolean raw) {
-        in.position(raw ? rawOff++ : off++);
-
-        return in.readBoolean();
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private BigDecimal doReadDecimal(boolean raw) {
-        int scale = doReadInt(raw);
-        byte[] mag = doReadByteArray(raw);
-
-        BigInteger intVal = new BigInteger(mag);
-
-        if (scale < 0) {
-            scale &= 0x7FFFFFFF;
-
-            intVal = intVal.negate();
-        }
-
-        return new BigDecimal(intVal, scale);
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private String doReadString(boolean raw) {
-        if (doReadBoolean(raw)) {
-            if (!in.hasArray())
-                return new String(doReadByteArray(raw), UTF_8);
-
-            int strLen = doReadInt(raw);
-            int strOff = raw ? rawOff : off;
-
-            String res = new String(in.array(), strOff, strLen, UTF_8);
-
-            if (raw)
-                rawOff += strLen;
-            else
-                off += strLen;
-
-            return res;
-        }
-        else
-            return String.valueOf(doReadCharArray(raw));
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private UUID doReadUuid(boolean raw) {
-        return new UUID(doReadLong(raw), doReadLong(raw));
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private Date doReadDate(boolean raw) {
-        long time = doReadLong(raw);
-
-        return new Date(time);
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private Timestamp doReadTimestamp(boolean raw) {
-        long time = doReadLong(raw);
-        int nanos = doReadInt(raw);
-
-        Timestamp ts = new Timestamp(time);
-
-        ts.setNanos(ts.getNanos() + nanos);
-
-        return ts;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable private Object doReadObject(boolean raw) throws PortableException {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, in, raw ? rawOff : off, ldr, rCtx);
-
-        Object obj = reader.deserialize();
-
-        if (raw)
-            rawOff += reader.len;
-        else
-            off += reader.len;
-
-        return obj;
-    }
-
-    /**
-     * @return Deserialized object.
-     * @throws PortableException If failed.
-     */
-    @Nullable Object deserialize() throws PortableException {
-        Object obj;
-
-        byte flag = doReadByte(true);
-
-        switch (flag) {
-            case NULL:
-                obj = null;
-
-                break;
-
-            case HANDLE:
-                int handle = start - doReadInt(true);
-
-                obj = rCtx.getObjectByHandle(handle);
-
-                if (obj == null) {
-                    off = handle;
-
-                    obj = doReadObject(false);
-                }
-
-                break;
-
-            case OBJ:
-                if (typeId == null)
-                    readObjectTypeId(true);
-
-                assert typeId != UNREGISTERED_TYPE_ID;
-
-                PortableUtils.checkProtocolVersion(doReadByte(true));
-
-                boolean userType = doReadBoolean(true);
-
-                // Skip typeId and hash code.
-                rawOff += 8;
-
-                desc = ctx.descriptorForTypeId(userType, typeId, ldr);
-
-                len = doReadInt(true);
-
-                rawOff = start + doReadInt(true);
-
-                if (desc == null)
-                    throw new PortableInvalidClassException("Unknown type ID: " + typeId);
-
-                obj = desc.read(this);
-
-                break;
-
-            case BYTE:
-                obj = doReadByte(true);
-
-                break;
-
-            case SHORT:
-                obj = doReadShort(true);
-
-                break;
-
-            case INT:
-                obj = doReadInt(true);
-
-                break;
-
-            case LONG:
-                obj = doReadLong(true);
-
-                break;
-
-            case FLOAT:
-                obj = doReadFloat(true);
-
-                break;
-
-            case DOUBLE:
-                obj = doReadDouble(true);
-
-                break;
-
-            case CHAR:
-                obj = doReadChar(true);
-
-                break;
-
-            case BOOLEAN:
-                obj = doReadBoolean(true);
-
-                break;
-
-            case DECIMAL:
-                obj = doReadDecimal(true);
-
-                break;
-
-            case STRING:
-                obj = doReadString(true);
-
-                break;
-
-            case UUID:
-                obj = doReadUuid(true);
-
-                break;
-
-            case DATE:
-                obj = doReadDate(true);
-
-                break;
-
-            case TIMESTAMP:
-                obj = doReadTimestamp(true);
-
-                break;
-
-            case BYTE_ARR:
-                obj = doReadByteArray(true);
-
-                break;
-
-            case SHORT_ARR:
-                obj = doReadShortArray(true);
-
-                break;
-
-            case INT_ARR:
-                obj = doReadIntArray(true);
-
-                break;
-
-            case LONG_ARR:
-                obj = doReadLongArray(true);
-
-                break;
-
-            case FLOAT_ARR:
-                obj = doReadFloatArray(true);
-
-                break;
-
-            case DOUBLE_ARR:
-                obj = doReadDoubleArray(true);
-
-                break;
-
-            case CHAR_ARR:
-                obj = doReadCharArray(true);
-
-                break;
-
-            case BOOLEAN_ARR:
-                obj = doReadBooleanArray(true);
-
-                break;
-
-            case DECIMAL_ARR:
-                obj = doReadDecimalArray(true);
-
-                break;
-
-            case STRING_ARR:
-                obj = doReadStringArray(true);
-
-                break;
-
-            case UUID_ARR:
-                obj = doReadUuidArray(true);
-
-                break;
-
-            case DATE_ARR:
-                obj = doReadDateArray(true);
-
-                break;
-
-            case TIMESTAMP_ARR:
-                obj = doReadTimestampArray(true);
-
-                break;
-
-            case OBJ_ARR:
-                obj = doReadObjectArray(true, true);
-
-                break;
-
-            case COL:
-                obj = doReadCollection(true, true, null);
-
-                break;
-
-            case MAP:
-                obj = doReadMap(true, true, null);
-
-                break;
-
-            case MAP_ENTRY:
-                obj = doReadMapEntry(true, true);
-
-                break;
-
-            case PORTABLE_OBJ:
-                obj = doReadPortableObject(true);
-
-                ((PortableObjectImpl)obj).context(ctx);
-
-                if (!GridPortableMarshaller.KEEP_PORTABLES.get())
-                    obj = ((PortableObject)obj).deserialize();
-
-                break;
-
-            case ENUM:
-                obj = doReadEnum(true, doReadClass(true));
-
-                break;
-
-            case ENUM_ARR:
-                obj = doReadEnumArray(true, doReadClass(true));
-
-                break;
-
-            case CLASS:
-                obj = doReadClass(true);
-
-                break;
-
-            case OPTM_MARSH:
-                int len = doReadInt(true);
-
-                ByteArrayInputStream input = new ByteArrayInputStream(in.array(), in.position(), len);
-
-                try {
-                    obj = ctx.optimizedMarsh().unmarshal(input, null);
-                }
-                catch (IgniteCheckedException e) {
-                    throw new PortableException("Failed to unmarshal object with optimized marshaller", e);
-                }
-
-                rawOff += len;
-
-                break;
-
-            default:
-                throw new PortableException("Invalid flag value: " + flag);
-        }
-
-        if (len == 0)
-            len = rawOff - start;
-
-        return obj;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private byte[] doReadByteArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        byte[] arr = in.readByteArray(len);
-
-        setHandler(arr, hPos);
-
-        if (raw)
-            rawOff += len;
-        else
-            off += len;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private short[] doReadShortArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        short[] arr = in.readShortArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 1;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private int[] doReadIntArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        int[] arr = in.readIntArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 2;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private long[] doReadLongArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        long[] arr = in.readLongArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 3;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private float[] doReadFloatArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        float[] arr = in.readFloatArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 2;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private double[] doReadDoubleArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        double[] arr = in.readDoubleArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 3;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private char[] doReadCharArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        char[] arr = in.readCharArray(len);
-
-        setHandler(arr, hPos);
-
-        int bytes = len << 1;
-
-        if (raw)
-            rawOff += bytes;
-        else
-            off += bytes;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private boolean[] doReadBooleanArray(boolean raw) {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        in.position(raw ? rawOff : off);
-
-        boolean[] arr = in.readBooleanArray(len);
-
-        setHandler(arr, hPos);
-
-        if (raw)
-            rawOff += len;
-        else
-            off += len;
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private BigDecimal[] doReadDecimalArray(boolean raw) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        BigDecimal[] arr = new BigDecimal[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else {
-                if (flag != DECIMAL)
-                    throw new PortableException("Invalid flag value: " + flag);
-
-                arr[i] = doReadDecimal(raw);
-            }
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private String[] doReadStringArray(boolean raw) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        String[] arr = new String[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else {
-                if (flag != STRING)
-                    throw new PortableException("Invalid flag value: " + flag);
-
-                arr[i] = doReadString(raw);
-            }
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private UUID[] doReadUuidArray(boolean raw) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        UUID[] arr = new UUID[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else {
-                if (flag != UUID)
-                    throw new PortableException("Invalid flag value: " + flag);
-
-                arr[i] = doReadUuid(raw);
-            }
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private Date[] doReadDateArray(boolean raw) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        Date[] arr = new Date[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else {
-                if (flag != DATE)
-                    throw new PortableException("Invalid flag value: " + flag);
-
-                arr[i] = doReadDate(raw);
-            }
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private Timestamp[] doReadTimestampArray(boolean raw) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int len = doReadInt(raw);
-
-        Timestamp[] arr = new Timestamp[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else {
-                if (flag != TIMESTAMP)
-                    throw new PortableException("Invalid flag value: " + flag);
-
-                arr[i] = doReadTimestamp(raw);
-            }
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param deep Deep flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private Object[] doReadObjectArray(boolean raw, boolean deep) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        Class compType = doReadClass(raw);
-
-        int len = doReadInt(raw);
-
-        Object[] arr = deep ? (Object[])Array.newInstance(compType, len) : new Object[len];
-
-        setHandler(arr, hPos);
-
-        for (int i = 0; i < len; i++)
-            arr[i] = deep ? doReadObject(raw) : unmarshal(raw);
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param deep Deep flag.
-     * @param cls Collection class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @SuppressWarnings("unchecked")
-    private Collection<?> doReadCollection(boolean raw, boolean deep, @Nullable Class<? extends Collection> cls)
-        throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int size = doReadInt(raw);
-
-        assert size >= 0;
-
-        byte colType = doReadByte(raw);
-
-        Collection<Object> col;
-
-        if (cls != null) {
-            try {
-                Constructor<? extends Collection> cons = cls.getConstructor();
-
-                col = cons.newInstance();
-            }
-            catch (NoSuchMethodException ignored) {
-                throw new PortableException("Collection class doesn't have public default constructor: " +
-                    cls.getName());
-            }
-            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
-                throw new PortableException("Failed to instantiate collection: " + cls.getName(), e);
-            }
-        }
-        else {
-            switch (colType) {
-                case ARR_LIST:
-                    col = new ArrayList<>(size);
-
-                    break;
-
-                case LINKED_LIST:
-                    col = new LinkedList<>();
-
-                    break;
-
-                case HASH_SET:
-                    col = U.newHashSet(size);
-
-                    break;
-
-                case LINKED_HASH_SET:
-                    col = U.newLinkedHashSet(size);
-
-                    break;
-
-                case TREE_SET:
-                    col = new TreeSet<>();
-
-                    break;
-
-                case CONC_SKIP_LIST_SET:
-                    col = new ConcurrentSkipListSet<>();
-
-                    break;
-
-                case USER_SET:
-                    col = U.newHashSet(size);
-
-                    break;
-
-                case USER_COL:
-                    col = new ArrayList<>(size);
-
-                    break;
-
-                default:
-                    throw new PortableException("Invalid collection type: " + colType);
-            }
-        }
-
-        setHandler(col, hPos);
-
-        for (int i = 0; i < size; i++)
-            col.add(deep ? doReadObject(raw) : unmarshal(raw));
-
-        return col;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param deep Deep flag.
-     * @param cls Map class.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    @SuppressWarnings("unchecked")
-    private Map<?, ?> doReadMap(boolean raw, boolean deep, @Nullable Class<? extends Map> cls)
-        throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        int size = doReadInt(raw);
-
-        assert size >= 0;
-
-        byte mapType = doReadByte(raw);
-
-        Map<Object, Object> map;
-
-        if (cls != null) {
-            try {
-                Constructor<? extends Map> cons = cls.getConstructor();
-
-                map = cons.newInstance();
-            }
-            catch (NoSuchMethodException ignored) {
-                throw new PortableException("Map class doesn't have public default constructor: " +
-                    cls.getName());
-            }
-            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
-                throw new PortableException("Failed to instantiate map: " + cls.getName(), e);
-            }
-        }
-        else {
-            switch (mapType) {
-                case HASH_MAP:
-                    map = U.newHashMap(size);
-
-                    break;
-
-                case LINKED_HASH_MAP:
-                    map = U.newLinkedHashMap(size);
-
-                    break;
-
-                case TREE_MAP:
-                    map = new TreeMap<>();
-
-                    break;
-
-                case CONC_HASH_MAP:
-                    map = new ConcurrentHashMap<>(size);
-
-                    break;
-
-                case USER_COL:
-                    map = U.newHashMap(size);
-
-                    break;
-
-                case PROPERTIES_MAP:
-                    map = new Properties();
-
-                    break;
-
-                default:
-                    throw new PortableException("Invalid map type: " + mapType);
-            }
-        }
-
-        setHandler(map, hPos);
-
-        for (int i = 0; i < size; i++)
-            map.put(deep ? doReadObject(raw) : unmarshal(raw), deep ? doReadObject(raw) : unmarshal(raw));
-
-        return map;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param deep Deep flag.
-     * @return Value.
-     * @throws PortableException In case of error.
-     */
-    private Map.Entry<?, ?> doReadMapEntry(boolean raw, boolean deep) throws PortableException {
-        int hPos = (raw ? rawOff : off) - 1;
-
-        Object val1 = deep ? doReadObject(raw) : unmarshal(raw);
-        Object val2 = deep ? doReadObject(raw) : unmarshal(raw);
-
-        GridMapEntry entry = new GridMapEntry<>(val1, val2);
-
-        setHandler(entry, hPos);
-
-        return entry;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private PortableObject doReadPortableObject(boolean raw) {
-        if (in.offheapPointer() > 0) {
-            int len = doReadInt(raw);
-
-            int pos = raw ? rawOff : off;
-
-            if (raw)
-                rawOff += len;
-            else
-                off += len;
-
-            int start = doReadInt(raw);
-
-            return new PortableObjectOffheapImpl(ctx, in.offheapPointer() + pos, start, len);
-        }
-        else {
-            byte[] arr = doReadByteArray(raw);
-            int start = doReadInt(raw);
-
-            return new PortableObjectImpl(ctx, arr, start);
-        }
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param cls Enum class.
-     * @return Value.
-     */
-    private Enum<?> doReadEnum(boolean raw, Class<?> cls) throws PortableException {
-        if (!cls.isEnum())
-            throw new PortableException("Class does not represent enum type: " + cls.getName());
-
-        int ord = doReadInt(raw);
-
-        return ord >= 0 ? (Enum<?>)GridEnumCache.get(cls)[ord] : null;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param cls Enum class.
-     * @return Value.
-     */
-    private Object[] doReadEnumArray(boolean raw, Class<?> cls) throws PortableException {
-        int len = doReadInt(raw);
-
-        Object[] arr = (Object[])Array.newInstance(cls, len);
-
-        for (int i = 0; i < len; i++) {
-            byte flag = doReadByte(raw);
-
-            if (flag == NULL)
-                arr[i] = null;
-            else
-                arr[i] = doReadEnum(raw, doReadClass(raw));
-        }
-
-        return arr;
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @return Value.
-     */
-    private Class doReadClass(boolean raw) throws PortableException {
-        return doReadClass(raw, doReadInt(raw));
-    }
-
-    /**
-     * @param raw Raw flag.
-     * @param typeId Type id.
-     * @return Value.
-     */
-    private Class doReadClass(boolean raw, int typeId) throws PortableException {
-        Class cls;
-
-        if (typeId == OBJECT_TYPE_ID)
-            return Object.class;
-
-        if (typeId != UNREGISTERED_TYPE_ID)
-            cls = ctx.descriptorForTypeId(true, typeId, ldr).describedClass();
-        else {
-            byte flag = doReadByte(raw);
-
-            if (flag != STRING)
-                throw new PortableException("No class definition for typeId: " + typeId);
-
-            String clsName = doReadString(raw);
-
-            try {
-                cls = U.forName(clsName, ldr);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            // forces registering of class by type id, at least locally
-            ctx.descriptorForClass(cls);
-        }
-
-        return cls;
-    }
-
-    /**
-     * @param name Field name.
-     * @return Field offset.
-     */
-    private int fieldId(String name) {
-        assert name != null;
-
-        if (typeId == null)
-            readObjectTypeId(false);
-
-        assert typeId != UNREGISTERED_TYPE_ID;
-
-        return ctx.fieldId(typeId, name);
-    }
-
-    /**
-     * @param id Field ID.
-     * @return Field offset.
-     */
-    private int fieldOffset(int id) {
-        assert hdrLen != 0;
-
-        int off = start + hdrLen;
-
-        int end = start + in.readInt(start + RAW_DATA_OFF_POS);
-
-        while (true) {
-            if (off >= end)
-                return -1;
-
-            int id0 = in.readInt(off);
-
-            if (id0 == id)
-                return off + 8;
-
-            int len = in.readInt(off + 4);
-
-            off += (8 + len);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readUnsignedByte() throws IOException {
-        return readByte() & 0xff;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readUnsignedShort() throws IOException {
-        return readShort() & 0xffff;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String readLine() throws IOException {
-        SB sb = new SB();
-
-        int b;
-
-        while ((b = read()) >= 0) {
-            char c = (char)b;
-
-            switch (c) {
-                case '\n':
-                    return sb.toString();
-
-                case '\r':
-                    b = read();
-
-                    if (b < 0 || b == '\n')
-                        return sb.toString();
-                    else
-                        sb.a((char)b);
-
-                    break;
-
-                default:
-                    sb.a(c);
-            }
-        }
-
-        return sb.toString();
-    }
-
-    /** {@inheritDoc} */
-    @NotNull @Override public String readUTF() throws IOException {
-        return readString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(byte[] b) throws IOException {
-        readFully(b, 0, b.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(byte[] b, int off, int len) throws IOException {
-        in.position(rawOff);
-
-        int cnt = in.read(b, off, len);
-
-        if (cnt < len)
-            throw new EOFException();
-
-        rawOff += len;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int skipBytes(int n) throws IOException {
-        int toSkip = Math.min(in.remaining(), n);
-
-        in.position(in.position() + toSkip);
-
-        rawOff += toSkip;
-
-        return toSkip;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read() throws IOException {
-        return readByte();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read(byte[] b) throws IOException {
-        return read(b, 0, b.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read(byte[] b, int off, int len) throws IOException {
-        in.position(rawOff);
-
-        int cnt = in.read(b, off, len);
-
-        rawOff += len;
-
-        return cnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long skip(long n) throws IOException {
-        return skipBytes((int)n);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int available() throws IOException {
-        return in.remaining();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        // No-op.
-    }
-}


[02/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
index 2f7bdb0..360ecec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/GridDataStreamerImplSelfTest.java
@@ -31,11 +31,11 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -221,12 +221,12 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
             }
 
             // Read random keys. Take values as PortableObject.
-            IgniteCache<Integer, PortableObject> c2 = ((IgniteCacheProxy)c).keepPortable();
+            IgniteCache<Integer, IgniteObject> c2 = ((IgniteCacheProxy)c).keepPortable();
 
             for (int i = 0; i < 100; i ++) {
                 Integer k = rnd.nextInt(KEYS_COUNT);
 
-                PortableObject v = c2.get(k);
+                IgniteObject v = c2.get(k);
 
                 assertEquals(k, v.field("val"));
             }
@@ -256,7 +256,7 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class TestObject implements PortableMarshalAware, Serializable {
+    private static class TestObject implements IgniteObjectMarshalAware, Serializable {
         /** */
         private int val;
 
@@ -289,19 +289,19 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("val", val);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val = reader.readInt("val");
         }
     }
 
     /**
      */
-    private static class TestObject2 implements PortableMarshalAware, Serializable {
+    private static class TestObject2 implements IgniteObjectMarshalAware, Serializable {
         /** */
         private int val;
 
@@ -333,12 +333,12 @@ public class GridDataStreamerImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("val", val);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val = reader.readInt("val");
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
index 155ba48..9964168 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAffinityRoutingPortableSelfTest.java
@@ -18,10 +18,11 @@
 package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 
 import java.util.Collections;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAffinityRoutingSelfTest;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
 
 /**
  *
@@ -31,10 +32,13 @@ public class GridCacheAffinityRoutingPortableSelfTest extends GridCacheAffinityR
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration();
 
         typeCfg.setClassName(AffinityTestKey.class.getName());
-        typeCfg.setAffinityKeyFieldName("affKey");
+
+        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(AffinityTestKey.class.getName(), "affKey");
+
+        cfg.setCacheKeyCfg(keyCfg);
 
         PortableMarshaller marsh = new PortableMarshaller();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
index 3f8cd1c..21a2085 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractDataStreamerSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractDataStreamerSelfTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -29,7 +29,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  *
  */
 public class GridCacheAtomicPartitionedOnlyPortableDataStreamerMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractDataStreamerSelfTest {
+    GridCacheIgniteObjectsAbstractDataStreamerSelfTest {
     /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
index 8f3a05f..5033204 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractMultiThreadedSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractMultiThreadedSelfTest;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -29,7 +29,7 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  *
  */
 public class GridCacheAtomicPartitionedOnlyPortableMultithreadedSelfTest extends
-    GridCachePortableObjectsAbstractMultiThreadedSelfTest {
+    GridCacheIgniteObjectsAbstractMultiThreadedSelfTest {
     /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return PARTITIONED;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.java
new file mode 100644
index 0000000..074634a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest
+    extends GridCacheIgniteObjectsAtomicNearDisabledSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean offheapTiered() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledSelfTest.java
new file mode 100644
index 0000000..c2ce87d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicNearDisabledSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsAtomicNearDisabledSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.java
new file mode 100644
index 0000000..f12cb8f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsAtomicOffheapTieredSelfTest extends GridCacheIgniteObjectsAtomicSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean offheapTiered() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicSelfTest.java
new file mode 100644
index 0000000..8601645
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsAtomicSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsAtomicSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
new file mode 100644
index 0000000..9c91802
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest
+    extends GridCacheIgniteObjectsPartitionedNearDisabledSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean offheapTiered() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.java
new file mode 100644
index 0000000..f8ff21c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsPartitionedNearDisabledSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.java
new file mode 100644
index 0000000..613d77e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.java
@@ -0,0 +1,30 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest
+    extends GridCacheIgniteObjectsPartitionedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean offheapTiered() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
new file mode 100644
index 0000000..c708f1b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheIgniteObjectsPartitionedSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.distributed.dht;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsPartitionedSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
index 1a0d601..b45a88a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionAtomicPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionAtomicSelfTest;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -66,7 +66,7 @@ public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCa
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }
@@ -87,7 +87,7 @@ public class GridCacheOffHeapTieredEvictionAtomicPortableSelfTest extends GridCa
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
index 60eed45..838109d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCacheOffHeapTieredEvictionPortableSelfTest.java
@@ -21,7 +21,7 @@ import java.util.Arrays;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheOffHeapTieredEvictionSelfTest;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -66,7 +66,7 @@ public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOff
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }
@@ -87,7 +87,7 @@ public class GridCacheOffHeapTieredEvictionPortableSelfTest extends GridCacheOff
 
         /** {@inheritDoc} */
         @Override public void checkValue(Object val) {
-            PortableObject obj = (PortableObject)val;
+            IgniteObject obj = (IgniteObject)val;
 
             assertEquals(expVal, obj.field("val"));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java
deleted file mode 100644
index a5c28f3..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest
-    extends GridCachePortableObjectsAtomicNearDisabledSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledSelfTest.java
deleted file mode 100644
index 696c3ed..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicNearDisabledSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsAtomicNearDisabledSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java
deleted file mode 100644
index 8e04fa1..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicOffheapTieredSelfTest.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsAtomicOffheapTieredSelfTest extends GridCachePortableObjectsAtomicSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicSelfTest.java
deleted file mode 100644
index 106e59b..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsAtomicSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsAtomicSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
deleted file mode 100644
index 5bc4672..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest
-    extends GridCachePortableObjectsPartitionedNearDisabledSelfTest{
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
deleted file mode 100644
index df55de7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedNearDisabledSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsPartitionedNearDisabledSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedOffheapTieredSelfTest.java
deleted file mode 100644
index a6bc0b4..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedOffheapTieredSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-/**
- *
- */
-public class GridCachePortableObjectsPartitionedOffheapTieredSelfTest
-    extends GridCachePortableObjectsPartitionedSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
deleted file mode 100644
index 8c248be..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/dht/GridCachePortableObjectsPartitionedSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.dht;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.PARTITIONED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsPartitionedSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCacheIgniteObjectsReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCacheIgniteObjectsReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCacheIgniteObjectsReplicatedSelfTest.java
new file mode 100644
index 0000000..76f32e8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCacheIgniteObjectsReplicatedSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.distributed.replicated;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsReplicatedSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 3;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
deleted file mode 100644
index 953fbfa..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/distributed/replicated/GridCachePortableObjectsReplicatedSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.distributed.replicated;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsReplicatedSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 3;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsAtomicLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsAtomicLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsAtomicLocalSelfTest.java
new file mode 100644
index 0000000..b4ba284
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsAtomicLocalSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.processors.cache.portable.local;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsAtomicLocalSelfTest extends GridCacheIgniteObjectsLocalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalOffheapTieredSelfTest.java
new file mode 100644
index 0000000..4a8c946
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalOffheapTieredSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.processors.cache.portable.local;
+
+/**
+ *
+ */
+public class GridCacheIgniteObjectsLocalOffheapTieredSelfTest extends GridCacheIgniteObjectsLocalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean offheapTiered() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
new file mode 100644
index 0000000..a3c5373
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCacheIgniteObjectsLocalSelfTest.java
@@ -0,0 +1,51 @@
+/*
+ * 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.processors.cache.portable.local;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheIgniteObjectsAbstractSelfTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.LOCAL;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public class GridCacheIgniteObjectsLocalSelfTest extends GridCacheIgniteObjectsAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return LOCAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 1;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsAtomicLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsAtomicLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsAtomicLocalSelfTest.java
deleted file mode 100644
index 3f3a350..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsAtomicLocalSelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.local;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-
-/**
- *
- */
-public class GridCachePortableObjectsAtomicLocalSelfTest extends GridCachePortableObjectsLocalSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java
deleted file mode 100644
index 53713ce..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalOffheapTieredSelfTest.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.ignite.internal.processors.cache.portable.local;
-
-/**
- *
- */
-public class GridCachePortableObjectsLocalOffheapTieredSelfTest extends GridCachePortableObjectsLocalSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean offheapTiered() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
deleted file mode 100644
index 1a87865..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/local/GridCachePortableObjectsLocalSelfTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable.local;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.portable.GridCachePortableObjectsAbstractSelfTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMode.LOCAL;
-
-/**
- * Test for portable objects stored in cache.
- */
-public class GridCachePortableObjectsLocalSelfTest extends GridCachePortableObjectsAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return LOCAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return TRANSACTIONAL;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 1;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
index 0e8b825..bd15f1f 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformComputePortableArgTask.java
@@ -25,8 +25,8 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskAdapter;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -87,9 +87,9 @@ public class PlatformComputePortableArgTask extends ComputeTaskAdapter<Object, I
 
         /** {@inheritDoc} */
         @Nullable @Override public Object execute() {
-            PortableObject arg0 = ((PortableObject)arg);
+            IgniteObject arg0 = ((IgniteObject)arg);
 
-            PortableMetadata meta = ignite.portables().metadata(arg0.typeId());
+            IgniteObjectMetadata meta = ignite.portables().metadata(arg0.typeId());
 
             if (meta == null)
                 throw new IgniteException("Metadata doesn't exist.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
index d9dee9d..f4e3ae8 100644
--- a/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformEventsWriteEventTask.java
@@ -32,7 +32,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.JobEvent;
 import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -41,8 +41,6 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.events.*;
 
-import org.apache.ignite.plugin.security.SecurityPermission;
-import org.apache.ignite.plugin.security.SecuritySubjectType;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.jetbrains.annotations.Nullable;
 
@@ -97,7 +95,7 @@ public class PlatformEventsWriteEventTask extends ComputeTaskAdapter<Long, Objec
 
             try (PlatformMemory mem = ctx.memory().get(ptr)) {
                 PlatformOutputStream out = mem.output();
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 int evtType = EventType.EVT_SWAP_SPACE_CLEARED;
                 String msg = "msg";

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index 1e4c828..a4a05aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -35,7 +35,7 @@ import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -271,7 +271,7 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
index d44f9c8..d2e17d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteCacheProcessProxy.java
@@ -424,7 +424,7 @@ public class IgniteCacheProcessProxy<K, V> implements IgniteCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
+    @Override public <K1, V1> IgniteCache<K1, V1> withKeepBinary() {
         throw new UnsupportedOperationException("Method should be supported.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 35e2dde..6c3d91d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -41,7 +41,7 @@ import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteIllegalStateException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -546,7 +546,7 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
index ecd25e1..01bc296 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePortableObjectsTestSuite.java
@@ -19,31 +19,31 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.portable.GridPortableAffinityKeySelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderStringAsCharsAdditionalSelfTest;
-import org.apache.ignite.internal.portable.GridPortableBuilderStringAsCharsSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderAdditionalSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest;
+import org.apache.ignite.internal.portable.GridIgniteObjectBuilderStringAsCharsSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMarshallerCtxDisabledSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMarshallerSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataDisabledSelfTest;
 import org.apache.ignite.internal.portable.GridPortableMetaDataSelfTest;
 import org.apache.ignite.internal.portable.GridPortableWildcardsSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataMultinodeTest;
-import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodePortableMetadataTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeIgniteObjectMetadataMultinodeTest;
+import org.apache.ignite.internal.processors.cache.portable.GridCacheClientNodeIgniteObjectMetadataTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStoreObjectsSelfTest;
 import org.apache.ignite.internal.processors.cache.portable.GridCachePortableStorePortablesSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicNearDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedNearDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCachePortableObjectsPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.distributed.replicated.GridCachePortableObjectsReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsAtomicLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsLocalOffheapTieredSelfTest;
-import org.apache.ignite.internal.processors.cache.portable.local.GridCachePortableObjectsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicNearDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedNearDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.dht.GridCacheIgniteObjectsPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.distributed.replicated.GridCacheIgniteObjectsReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsAtomicLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsLocalOffheapTieredSelfTest;
+import org.apache.ignite.internal.processors.cache.portable.local.GridCacheIgniteObjectsLocalSelfTest;
 
 /**
  * Test for portable objects stored in cache.
@@ -58,34 +58,34 @@ public class IgnitePortableObjectsTestSuite extends TestSuite {
 
         suite.addTestSuite(GridPortableMarshallerSelfTest.class);
         suite.addTestSuite(GridPortableMarshallerCtxDisabledSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderStringAsCharsSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderStringAsCharsSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataSelfTest.class);
         suite.addTestSuite(GridPortableMetaDataDisabledSelfTest.class);
         suite.addTestSuite(GridPortableAffinityKeySelfTest.class);
         suite.addTestSuite(GridPortableWildcardsSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderAdditionalSelfTest.class);
-        suite.addTestSuite(GridPortableBuilderStringAsCharsAdditionalSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderAdditionalSelfTest.class);
+        suite.addTestSuite(GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.class);
 
-        suite.addTestSuite(GridCachePortableObjectsLocalSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicLocalSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsReplicatedSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedNearDisabledSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicNearDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsLocalSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicLocalSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedNearDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicNearDisabledSelfTest.class);
 
-        suite.addTestSuite(GridCachePortableObjectsLocalOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsAtomicNearDisabledOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedOffheapTieredSelfTest.class);
-        suite.addTestSuite(GridCachePortableObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsLocalOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsAtomicNearDisabledOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedOffheapTieredSelfTest.class);
+        suite.addTestSuite(GridCacheIgniteObjectsPartitionedNearDisabledOffheapTieredSelfTest.class);
 
         suite.addTestSuite(GridCachePortableStoreObjectsSelfTest.class);
         suite.addTestSuite(GridCachePortableStorePortablesSelfTest.class);
 
-        suite.addTestSuite(GridCacheClientNodePortableMetadataTest.class);
-        suite.addTestSuite(GridCacheClientNodePortableMetadataMultinodeTest.class);
+        suite.addTestSuite(GridCacheClientNodeIgniteObjectMetadataTest.class);
+        suite.addTestSuite(GridCacheClientNodeIgniteObjectMetadataMultinodeTest.class);
 
         return suite;
     }


[14/19] ignite git commit: ignite-950-new WIP

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
index e03d67f..1db7fb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataHandler.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
 
 /**
  * Portable meta data handler.
@@ -29,16 +29,16 @@ public interface PortableMetaDataHandler {
      *
      * @param typeId Type ID.
      * @param meta Meta data.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public void addMeta(int typeId, PortableMetadata meta) throws PortableException;
+    public void addMeta(int typeId, IgniteObjectMetadata meta) throws IgniteObjectException;
 
     /**
      * Gets meta data for provided type ID.
      *
      * @param typeId Type ID.
      * @return Meta data.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public PortableMetadata metadata(int typeId) throws PortableException;
+    public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
deleted file mode 100644
index 1d26007..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableMetaDataImpl.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable meta data implementation.
- */
-public class PortableMetaDataImpl implements PortableMetadata, PortableMarshalAware, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private String typeName;
-
-    /** */
-    @GridToStringInclude
-    private Map<String, String> fields;
-
-    /** */
-    private volatile Map<Integer, String> fldIdToName;
-
-    /** */
-    private String affKeyFieldName;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public PortableMetaDataImpl() {
-        // No-op.
-    }
-
-    /**
-     * @param typeName Type name.
-     * @param fields Fields map.
-     * @param affKeyFieldName Affinity key field name.
-     */
-    public PortableMetaDataImpl(String typeName, @Nullable Map<String, String> fields,
-        @Nullable String affKeyFieldName) {
-        assert typeName != null;
-
-        this.typeName = typeName;
-        this.fields = fields;
-        this.affKeyFieldName = affKeyFieldName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String typeName() {
-        return typeName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<String> fields() {
-        return fields != null ? fields.keySet() : Collections.<String>emptyList();
-    }
-
-    /**
-     * @return Fields.
-     */
-    public Map<String, String> fields0() {
-        return fields != null ? fields : Collections.<String, String>emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String fieldTypeName(String fieldName) {
-        return fields != null ? fields.get(fieldName) : null;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String affinityKeyFieldName() {
-        return affKeyFieldName;
-    }
-
-    /**
-     * @return Fields meta data.
-     */
-    public Map<String, String> fieldsMeta() {
-        return fields != null ? fields : Collections.<String, String>emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeString(out, typeName);
-        U.writeMap(out, fields);
-        U.writeString(out, affKeyFieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        typeName = U.readString(in);
-        fields = U.readMap(in);
-        affKeyFieldName = U.readString(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        PortableRawWriter raw = writer.rawWriter();
-
-        raw.writeString(typeName);
-        raw.writeString(affKeyFieldName);
-        raw.writeMap(fields);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        PortableRawReader raw = reader.rawReader();
-
-        typeName = raw.readString();
-        affKeyFieldName = raw.readString();
-        fields = raw.readMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(PortableMetaDataImpl.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
deleted file mode 100644
index fe4b628..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectEx.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.math.BigDecimal;
-import java.util.Arrays;
-import java.util.IdentityHashMap;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Internal portable object interface.
- */
-public abstract class PortableObjectEx implements PortableObject {
-    /**
-     * @return Length.
-     */
-    public abstract int length();
-
-    /**
-     * @return Object start.
-     */
-    public abstract int start();
-
-    /**
-     * @return {@code True} if object is array based.
-     */
-    protected abstract boolean hasArray();
-
-    /**
-     * @return Object array if object is array based, otherwise {@code null}.
-     */
-    public abstract byte[] array();
-
-    /**
-     * @return Object offheap address is object is offheap based, otherwise 0.
-     */
-    public abstract long offheapAddress();
-
-    /**
-     * @param ctx Reader context.
-     * @param fieldName Field name.
-     * @return Field name.
-     */
-    @Nullable protected abstract <F> F field(PortableReaderContext ctx, String fieldName);
-
-    /** {@inheritDoc} */
-    @Override public PortableObject clone() throws CloneNotSupportedException {
-        return (PortableObject)super.clone();
-    }
-
-    /** {@inheritDoc} */
-    public boolean equals(Object other) {
-        if (other == this)
-            return true;
-
-        if (other == null)
-            return false;
-
-        if (!(other instanceof PortableObjectEx))
-            return false;
-
-        PortableObjectEx otherPo = (PortableObjectEx)other;
-
-        if (length() != otherPo.length() || typeId() != otherPo.typeId())
-            return false;
-
-        if (hasArray()) {
-            if (otherPo.hasArray()) {
-                int len = length();
-                int end = start() + len;
-
-                byte[] arr = array();
-                byte[] otherArr = otherPo.array();
-
-                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
-                    if (arr[i] != otherArr[j])
-                        return false;
-                }
-
-                return true;
-            }
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
-            }
-        }
-        else {
-            assert offheapAddress() > 0;
-
-            if (otherPo.hasArray())
-                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
-            else {
-                assert otherPo.offheapAddress() > 0;
-
-                return GridUnsafeMemory.compare(offheapAddress() + start(),
-                    otherPo.offheapAddress() + otherPo.start(),
-                    length());
-            }
-        }
-    }
-
-    /**
-     * @param ctx Reader context.
-     * @param handles Handles for already traversed objects.
-     * @return String representation.
-     */
-    private String toString(PortableReaderContext ctx, IdentityHashMap<PortableObject, Integer> handles) {
-        int idHash = System.identityHashCode(this);
-
-        PortableMetadata meta;
-
-        try {
-            meta = metaData();
-        }
-        catch (PortableException ignore) {
-            meta = null;
-        }
-
-        if (meta == null)
-            return "PortableObject [hash=" + idHash + ", typeId=" + typeId() + ']';
-
-        handles.put(this, idHash);
-
-        SB buf = new SB(meta.typeName());
-
-        if (meta.fields() != null) {
-            buf.a(" [hash=").a(idHash);
-
-            for (String name : meta.fields()) {
-                Object val = field(ctx, name);
-
-                buf.a(", ").a(name).a('=');
-
-                if (val instanceof byte[])
-                    buf.a(Arrays.toString((byte[]) val));
-                else if (val instanceof short[])
-                    buf.a(Arrays.toString((short[])val));
-                else if (val instanceof int[])
-                    buf.a(Arrays.toString((int[])val));
-                else if (val instanceof long[])
-                    buf.a(Arrays.toString((long[])val));
-                else if (val instanceof float[])
-                    buf.a(Arrays.toString((float[])val));
-                else if (val instanceof double[])
-                    buf.a(Arrays.toString((double[])val));
-                else if (val instanceof char[])
-                    buf.a(Arrays.toString((char[])val));
-                else if (val instanceof boolean[])
-                    buf.a(Arrays.toString((boolean[]) val));
-                else if (val instanceof BigDecimal[])
-                    buf.a(Arrays.toString((BigDecimal[])val));
-                else {
-                    if (val instanceof PortableObjectEx) {
-                        PortableObjectEx po = (PortableObjectEx)val;
-
-                        Integer idHash0 = handles.get(val);
-
-                        if (idHash0 != null) {  // Circular reference.
-                            PortableMetadata meta0 = po.metaData();
-
-                            assert meta0 != null;
-
-                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
-                        }
-                        else
-                            buf.a(po.toString(ctx, handles));
-                    }
-                    else
-                        buf.a(val);
-                }
-            }
-
-            buf.a(']');
-        }
-
-        return buf.toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        try {
-            PortableReaderContext ctx = new PortableReaderContext();
-
-            ctx.setPortableHandler(start(), this);
-
-            return toString(ctx, new IdentityHashMap<PortableObject, Integer>());
-        }
-        catch (PortableException e) {
-            throw new IgniteException("Failed to create string representation of portable object.", e);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
deleted file mode 100644
index d82162a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectImpl.java
+++ /dev/null
@@ -1,411 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.GridDirectTransient;
-import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.plugin.extensions.communication.Message;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable object implementation.
- */
-@IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator.
-public final class PortableObjectImpl extends PortableObjectEx implements Externalizable,
-    Message, CacheObject, KeyCacheObject {
-    /** */
-    public static final byte TYPE_PORTABLE = 100;
-
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final PortablePrimitives PRIM = PortablePrimitives.get();
-
-    /** */
-    @GridDirectTransient
-    private PortableContext ctx;
-
-    /** */
-    private byte[] arr;
-
-    /** */
-    private int start;
-
-    /** */
-    @GridDirectTransient
-    private Object obj;
-
-    /** */
-    @GridDirectTransient
-    private boolean detachAllowed;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public PortableObjectImpl() {
-        // No-op.
-    }
-
-    /**
-     * @param ctx Context.
-     * @param arr Array.
-     * @param start Start.
-     */
-    public PortableObjectImpl(PortableContext ctx, byte[] arr, int start) {
-        assert ctx != null;
-        assert arr != null;
-
-        this.ctx = ctx;
-        this.arr = arr;
-        this.start = start;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte type() {
-        return TYPE_PORTABLE;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isPlatformType() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean internal() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        Object obj0 = obj;
-
-        if (obj0 == null || cpy)
-            obj0 = deserializeValue();
-
-        return (T)obj0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
-        if (detached())
-            return array();
-
-        int len = length();
-
-        byte[] arr0 = new byte[len];
-
-        U.arrayCopy(arr, start, arr0, 0, len);
-
-        return arr0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        if (detached())
-            return this;
-
-        return (PortableObjectImpl)detach();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-        this.ctx = ((CacheObjectPortableProcessorImpl)ctx.processor()).portableContext();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public int length() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS);
-    }
-
-    /**
-     * @return Detached portable object.
-     */
-    public PortableObject detach() {
-        if (!detachAllowed || detached())
-            return this;
-
-        int len = length();
-
-        byte[] arr0 = new byte[len];
-
-        U.arrayCopy(arr, start, arr0, 0, len);
-
-        return new PortableObjectImpl(ctx, arr0, 0);
-    }
-
-    /**
-     * @return Detached or not.
-     */
-    public boolean detached() {
-        return start == 0 && length() == arr.length;
-    }
-
-    /**
-     * @return {@code True} if detach is allowed.
-     */
-    public boolean detachAllowed() {
-        return true;
-    }
-
-    /**
-     * @param detachAllowed Detach allowed flag.
-     */
-    public void detachAllowed(boolean detachAllowed) {
-        this.detachAllowed = detachAllowed;
-    }
-
-    /**
-     * @return Context.
-     */
-    public PortableContext context() {
-        return ctx;
-    }
-
-    /**
-     * @param ctx Context.
-     */
-    public void context(PortableContext ctx) {
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] array() {
-        return arr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int start() {
-        return start;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long offheapAddress() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean hasArray() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int typeId() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metaData() throws PortableException {
-        if (ctx == null)
-            throw new PortableException("PortableContext is not set for the object.");
-
-        return ctx.metaData(typeId());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <F> F field(String fieldName) throws PortableException {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx,
-            new PortableHeapInputStream(arr),
-            start,
-            null,
-            rCtx);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasField(String fieldName) {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, arr, start, null);
-
-        return reader.hasField(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T deserialize() throws PortableException {
-        Object obj0 = obj;
-
-        if (obj0 == null)
-            obj0 = deserializeValue();
-
-        return (T)obj0;
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject clone() throws CloneNotSupportedException {
-        return super.clone();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return PRIM.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(ctx);
-
-        if (detachAllowed) {
-            int len = length();
-
-            out.writeInt(len);
-            out.write(arr, start, len);
-            out.writeInt(0);
-        }
-        else {
-            out.writeInt(arr.length);
-            out.write(arr);
-            out.writeInt(start);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        ctx = (PortableContext)in.readObject();
-
-        arr = new byte[in.readInt()];
-
-        in.readFully(arr);
-
-        start = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType(), fieldsCount()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
-        switch (writer.state()) {
-            case 0:
-                if (!writer.writeByteArray("arr",
-                    arr,
-                    detachAllowed ? start : 0,
-                    detachAllowed ? length() : arr.length))
-                    return false;
-
-                writer.incrementState();
-
-            case 1:
-                if (!writer.writeInt("start", detachAllowed ? 0 : start))
-                    return false;
-
-                writer.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        if (!reader.beforeMessageRead())
-            return false;
-
-        switch (reader.state()) {
-            case 0:
-                arr = reader.readByteArray("arr");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 1:
-                start = reader.readInt("start");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        return 113;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        return 3;
-    }
-
-    /**
-     * Runs value deserialization regardless of whether obj already has the deserialized value.
-     * Will set obj if descriptor is configured to keep deserialized values.
-     */
-    private Object deserializeValue() {
-        // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, arr, start, null);
-
-        Object obj0 = reader.deserialize();
-
-        PortableClassDescriptor desc = reader.descriptor();
-
-        assert desc != null;
-
-        if (desc.keepDeserialized())
-            obj = obj0;
-
-        return obj0;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
deleted file mode 100644
index 6d8231b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableObjectOffheapImpl.java
+++ /dev/null
@@ -1,255 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.nio.ByteBuffer;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
-import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.CacheObjectContext;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-import sun.misc.Unsafe;
-
-/**
- *  Portable object implementation over offheap memory
- */
-public class PortableObjectOffheapImpl extends PortableObjectEx implements Externalizable, CacheObject {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final long ptr;
-
-    /** */
-    private final int start;
-
-    /** */
-    private final int size;
-
-    /**
-     * For {@link Externalizable} (not supported).
-     */
-    public PortableObjectOffheapImpl() {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @param ctx Context.
-     * @param ptr Memory address.
-     * @param start Object start.
-     * @param size Memory size.
-     */
-    public PortableObjectOffheapImpl(PortableContext ctx, long ptr, int start, int size) {
-        this.ctx = ctx;
-        this.ptr = ptr;
-        this.start = start;
-        this.size = size;
-    }
-
-    /**
-     * @return Heap-based copy.
-     */
-    public PortableObject heapCopy() {
-        return new PortableObjectImpl(ctx, U.copyMemory(ptr, size), start);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int typeId() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TYPE_ID_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int length() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TOTAL_LEN_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.HASH_CODE_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int start() {
-        return start;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] array() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long offheapAddress() {
-        return ptr;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean hasArray() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metaData() throws PortableException {
-        if (ctx == null)
-            throw new PortableException("PortableContext is not set for the object.");
-
-        return ctx.metaData(typeId());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <F> F field(String fieldName) throws PortableException {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null,
-            rCtx);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasField(String fieldName) {
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.hasField(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T deserialize() throws PortableException {
-        return (T)deserializeValue();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("CloneDoesntCallSuperClone")
-    @Override public PortableObject clone() throws CloneNotSupportedException {
-        return heapCopy();
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte type() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isPlatformType() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)deserializeValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        throw new UnsupportedOperationException(); // To make sure it is not marshalled.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        throw new UnsupportedOperationException(); // To make sure it is not marshalled.
-    }
-
-    /**
-     * @return Deserialized value.
-     */
-    private Object deserializeValue() {
-        // TODO: IGNITE-1272 - Deserialize with proper class loader.
-        PortableReaderExImpl reader = new PortableReaderExImpl(
-            ctx,
-            new PortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.deserialize();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
deleted file mode 100644
index e703f2f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawReaderEx.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawReader;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Extended reader interface.
- */
-public interface PortableRawReaderEx extends PortableRawReader {
-    /**
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Object readObjectDetached() throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
deleted file mode 100644
index a59f157..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableRawWriterEx.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Extended writer interface.
- */
-public interface PortableRawWriterEx extends PortableRawWriter, AutoCloseable {
-    /**
-     * @param obj Object to write.
-     * @throws PortableException In case of error.
-     */
-    public void writeObjectDetached(@Nullable Object obj) throws PortableException;
-
-    /**
-     * @return Output stream.
-     */
-    public PortableOutputStream out();
-
-    /**
-     * Cleans resources.
-     */
-    @Override public void close();
-
-    /**
-     * Reserve a room for an integer.
-     *
-     * @return Position in the stream where value is to be written.
-     */
-    public int reserveInt();
-
-    /**
-     * Write int value at the specific position.
-     *
-     * @param pos Position.
-     * @param val Value.
-     * @throws PortableException If failed.
-     */
-    public void writeInt(int pos, int val) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
index 2d4a1c3..bf1c3b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderContext.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.portable;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -31,7 +31,7 @@ class PortableReaderContext {
     private Map<Integer, Object> oHandles;
 
     /** */
-    private Map<Integer, PortableObject> poHandles;
+    private Map<Integer, IgniteObject> poHandles;
 
     /**
      * @param handle Handle.
@@ -50,7 +50,7 @@ class PortableReaderContext {
      * @param handle Handle.
      * @param po Portable object.
      */
-    void setPortableHandler(int handle, PortableObject po) {
+    void setPortableHandler(int handle, IgniteObject po) {
         assert po != null;
 
         if (poHandles == null)
@@ -71,7 +71,7 @@ class PortableReaderContext {
      * @param handle Handle.
      * @return Object.
      */
-    @Nullable PortableObject getPortableByHandle(int handle) {
+    @Nullable IgniteObject getPortableByHandle(int handle) {
         return poHandles != null ? poHandles.get(handle) : null;
     }
 


[16/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectReaderExImpl.java
new file mode 100644
index 0000000..b018f7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectReaderExImpl.java
@@ -0,0 +1,3230 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
+import org.apache.ignite.internal.portable.streams.PortableInputStream;
+import org.apache.ignite.internal.util.GridEnumCache;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectRawReader;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.ByteArrayInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.lang.reflect.Array;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Properties;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.ARR_LIST;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_HASH_MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CONC_SKIP_LIST_SET;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.HANDLE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_SET;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_HASH_MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_HASH_SET;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LINKED_LIST;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJECT_TYPE_ID;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROPERTIES_MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TREE_SET;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_COL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.USER_SET;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR;
+
+/**
+ * Portable reader implementation.
+ */
+@SuppressWarnings("unchecked")
+public class IgniteObjectReaderExImpl implements IgniteObjectReader, IgniteObjectRawReaderEx, ObjectInput {
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final PortableInputStream in;
+
+    /** */
+    private final int start;
+
+    /** */
+    private final PortableReaderContext rCtx;
+
+    /** */
+    private final ClassLoader ldr;
+
+    /** */
+    private int off;
+
+    /** */
+    private int rawOff;
+
+    /** */
+    private int len;
+
+    /** */
+    private PortableClassDescriptor desc;
+
+    /** */
+    private int hdrLen;
+
+    /** */
+    private int clsNameLen;
+
+    /** */
+    private Integer typeId;
+
+    /**
+     * @param ctx Context.
+     * @param arr Array.
+     * @param start Start.
+     * @param ldr Class loader.
+     */
+    public IgniteObjectReaderExImpl(PortableContext ctx, byte[] arr, int start, ClassLoader ldr) {
+        this(ctx, new PortableHeapInputStream(arr), start, ldr, new PortableReaderContext());
+    }
+
+    /**
+     * @param ctx Context.
+     * @param in Input stream.
+     * @param start Start.
+     */
+    IgniteObjectReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr) {
+        this(ctx, in, start, ldr, new PortableReaderContext());
+    }
+
+    /**
+     * @param ctx Context.
+     * @param in Input stream.
+     * @param start Start.
+     * @param rCtx Context.
+     */
+    IgniteObjectReaderExImpl(PortableContext ctx, PortableInputStream in, int start, ClassLoader ldr,
+        PortableReaderContext rCtx) {
+        this.ctx = ctx;
+        this.in = in;
+        this.start = start;
+        this.ldr = ldr;
+        this.rCtx = rCtx;
+
+        off = start;
+        rawOff = start;
+    }
+
+    /**
+     * Preloads typeId from the input array.
+     */
+    private void readObjectTypeId(boolean skipObjByte) {
+        int pos = rawOff;
+
+        if (!skipObjByte)
+            // skip obj type byte
+            rawOff++;
+
+        // Validate protocol version.
+        PortableUtils.checkProtocolVersion(doReadByte(true));
+
+        // skip user flag
+        rawOff += 1;
+
+        typeId = doReadInt(true);
+
+        if (typeId == UNREGISTERED_TYPE_ID) {
+            // skip hash code, length and raw offset
+            rawOff += 12;
+
+            int off = rawOff;
+
+            Class cls = doReadClass(true, typeId);
+
+            // registers class by typeId, at least locally if the cache is not ready yet.
+            PortableClassDescriptor desc = ctx.descriptorForClass(cls);
+
+            typeId = desc.typeId();
+
+            clsNameLen = rawOff - off;
+
+            hdrLen = CLS_NAME_POS + clsNameLen;
+        }
+        else
+            hdrLen = DFLT_HDR_LEN;
+
+        in.position(rawOff = pos);
+    }
+
+    /**
+     * @return Descriptor.
+     */
+    PortableClassDescriptor descriptor() {
+        return desc;
+    }
+
+    /**
+     * @return Unmarshalled value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Object unmarshal() throws IgniteObjectException {
+        return unmarshal(true);
+    }
+
+    /**
+     * @param fieldName Field name.
+     * @return Unmarshalled value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Object unmarshal(String fieldName) throws IgniteObjectException {
+        off = fieldOffset(fieldId(fieldName));
+
+        return off >= 0 ? unmarshal(false) : null;
+    }
+
+    /**
+     * @param offset Offset in the array.
+     * @return Unmarshalled value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public Object unmarshal(int offset) throws IgniteObjectException {
+        off = offset;
+
+        return off >= 0 ? unmarshal(false) : null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Byte readByte(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != BYTE)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadByte(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Short readShort(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != SHORT)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadShort(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Integer readInt(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != INT)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadInt(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Long readLong(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != LONG)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadLong(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Float readFloat(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != FLOAT)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadFloat(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Double readDouble(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != DOUBLE)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDouble(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Character readChar(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != CHAR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadChar(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Boolean readBoolean(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != BOOLEAN)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadBoolean(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable BigDecimal readDecimal(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != DECIMAL)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDecimal(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable String readString(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != STRING)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadString(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable UUID readUuid(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != UUID)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadUuid(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Date readDate(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != DATE)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDate(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Timestamp readTimestamp(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != TIMESTAMP)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadTimestamp(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Object readObject(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        return off >= 0 ? doReadObject(false) : null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable byte[] readByteArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != BYTE_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadByteArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable short[] readShortArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != SHORT_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadShortArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable int[] readIntArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != INT_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadIntArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable long[] readLongArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != LONG_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadLongArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable float[] readFloatArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != FLOAT_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadFloatArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable double[] readDoubleArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != DOUBLE_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDoubleArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable char[] readCharArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != CHAR_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadCharArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable boolean[] readBooleanArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != BOOLEAN_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadBooleanArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable BigDecimal[] readDecimalArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != DECIMAL_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDecimalArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable String[] readStringArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != STRING_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadStringArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable UUID[] readUuidArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != UUID_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadUuidArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Date[] readDateArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != DATE_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadDateArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Timestamp[] readTimestampArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != TIMESTAMP_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadTimestampArray(false);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Object[] readObjectArray(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != OBJ_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadObjectArray(false, true);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @param cls Collection class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable <T> Collection<T> readCollection(int fieldId, @Nullable Class<? extends Collection> cls)
+        throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != COL)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return (Collection<T>)doReadCollection(false, true, cls);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @param cls Map class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Map<?, ?> readMap(int fieldId, @Nullable Class<? extends Map> cls)
+        throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != MAP)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadMap(false, true, cls);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException On case of error.
+     */
+    @Nullable Map.Entry<?, ?> readMapEntry(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag == HANDLE)
+                return readHandleField();
+
+            if (flag != MAP_ENTRY)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return doReadMapEntry(false, true);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Portable object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable IgniteObject readPortableObject(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != PORTABLE_OBJ)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            return new IgniteObjectImpl(ctx, doReadByteArray(false), doReadInt(false));
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @param cls Class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Enum<?> readEnum(int fieldId, @Nullable Class<?> cls) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != ENUM)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            // Revisit: why have we started writing Class for enums in their serialized form?
+            if (cls == null)
+                cls = doReadClass(false);
+            else
+                doReadClass(false);
+
+            Object[] vals = GridEnumCache.get(cls);
+
+            return (Enum<?>)vals[doReadInt(false)];
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @param cls Class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Object[] readEnumArray(int fieldId, @Nullable Class<?> cls) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != ENUM_ARR)
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+
+            // Revisit: why have we started writing Class for enums in their serialized form?
+            if (cls == null)
+                cls = doReadClass(false);
+            else
+                doReadClass(false);
+
+            return doReadEnumArray(false, cls);
+        }
+        else
+            return null;
+    }
+
+    /**
+     * @param fieldId Field ID.
+     * @return Field class.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable Class<?> readClass(int fieldId) throws IgniteObjectException {
+        off = fieldOffset(fieldId);
+
+        if (off >= 0) {
+            byte flag = doReadByte(false);
+
+            if (flag == NULL)
+                return null;
+
+            if (flag != CLASS)
+                throw new IgniteObjectException("Invalid flag type: [flag=" + flag + ']');
+
+            return doReadClass(false);
+        }
+
+        return null;
+    }
+
+    /**
+     * @param obj Object.
+     */
+    void setHandler(Object obj) {
+        rCtx.setObjectHandler(start, obj);
+    }
+
+    /**
+     * @param obj Object.
+     * @param pos Position.
+     */
+    void setHandler(Object obj, int pos) {
+        rCtx.setObjectHandler(pos, obj);
+    }
+
+    /**
+     * Recreating field value from a handle.
+     *
+     * @param <T> Field type.
+     * @return Field.
+     */
+    private <T> T readHandleField() {
+        int handle = (off - 1) - doReadInt(false);
+
+        Object obj = rCtx.getObjectByHandle(handle);
+
+        if (obj == null) {
+            off = handle;
+
+            obj = doReadObject(false);
+        }
+
+        return (T)obj;
+    }
+    /** {@inheritDoc} */
+    @Override public byte readByte(String fieldName) throws IgniteObjectException {
+        Byte val = readByte(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte readByte() throws IgniteObjectException {
+        return doReadByte(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort(String fieldName) throws IgniteObjectException {
+        Short val = readShort(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort() throws IgniteObjectException {
+        return doReadShort(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt(String fieldName) throws IgniteObjectException {
+        Integer val = readInt(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt() throws IgniteObjectException {
+        return doReadInt(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong(String fieldName) throws IgniteObjectException {
+        Long val = readLong(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong() throws IgniteObjectException {
+        return doReadLong(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat(String fieldName) throws IgniteObjectException {
+        Float val = readFloat(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat() throws IgniteObjectException {
+        return doReadFloat(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble(String fieldName) throws IgniteObjectException {
+        Double val = readDouble(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble() throws IgniteObjectException {
+        return doReadDouble(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar(String fieldName) throws IgniteObjectException {
+        Character val = readChar(fieldId(fieldName));
+
+        return val != null ? val : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar() throws IgniteObjectException {
+        return doReadChar(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean(String fieldName) throws IgniteObjectException {
+        Boolean val = readBoolean(fieldId(fieldName));
+
+        return val != null ? val : false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean() throws IgniteObjectException {
+        return doReadBoolean(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public BigDecimal readDecimal(String fieldName) throws IgniteObjectException {
+        return readDecimal(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public BigDecimal readDecimal() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != DECIMAL)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadDecimal(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String readString(String fieldName) throws IgniteObjectException {
+        return readString(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String readString() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != STRING)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadString(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID readUuid(String fieldName) throws IgniteObjectException {
+        return readUuid(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID readUuid() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != UUID)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadUuid(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Date readDate(String fieldName) throws IgniteObjectException {
+        return readDate(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Date readDate() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != DATE)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadDate(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Timestamp readTimestamp(String fieldName) throws IgniteObjectException {
+        return readTimestamp(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Timestamp readTimestamp() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != TIMESTAMP)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadTimestamp(true);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T readObject(String fieldName) throws IgniteObjectException {
+        return (T)readObject(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object readObject() throws IgniteObjectException {
+        return doReadObject(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object readObjectDetached() throws IgniteObjectException {
+        return unmarshal(true, true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] readByteArray(String fieldName) throws IgniteObjectException {
+        return readByteArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] readByteArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != BYTE_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadByteArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public short[] readShortArray(String fieldName) throws IgniteObjectException {
+        return readShortArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public short[] readShortArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != SHORT_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadShortArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public int[] readIntArray(String fieldName) throws IgniteObjectException {
+        return readIntArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public int[] readIntArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != INT_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadIntArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public long[] readLongArray(String fieldName) throws IgniteObjectException {
+        return readLongArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public long[] readLongArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != LONG_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadLongArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public float[] readFloatArray(String fieldName) throws IgniteObjectException {
+        return readFloatArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public float[] readFloatArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != FLOAT_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadFloatArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public double[] readDoubleArray(String fieldName) throws IgniteObjectException {
+        return readDoubleArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public double[] readDoubleArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != DOUBLE_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadDoubleArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public char[] readCharArray(String fieldName) throws IgniteObjectException {
+        return readCharArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public char[] readCharArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != CHAR_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadCharArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public boolean[] readBooleanArray(String fieldName) throws IgniteObjectException {
+        return readBooleanArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public boolean[] readBooleanArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != BOOLEAN_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadBooleanArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws IgniteObjectException {
+        return readDecimalArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public BigDecimal[] readDecimalArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != DECIMAL_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadDecimalArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String[] readStringArray(String fieldName) throws IgniteObjectException {
+        return readStringArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String[] readStringArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != STRING_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadStringArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID[] readUuidArray(String fieldName) throws IgniteObjectException {
+        return readUuidArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID[] readUuidArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != UUID_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadUuidArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Date[] readDateArray(String fieldName) throws IgniteObjectException {
+        return readDateArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Timestamp[] readTimestampArray(String fieldName) throws IgniteObjectException {
+        return readTimestampArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Date[] readDateArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != DATE_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadDateArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Timestamp[] readTimestampArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != TIMESTAMP_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadTimestampArray(true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object[] readObjectArray(String fieldName) throws IgniteObjectException {
+        return readObjectArray(fieldId(fieldName));
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object[] readObjectArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != OBJ_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return doReadObjectArray(true, true);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection(String fieldName) throws IgniteObjectException {
+        return readCollection(fieldId(fieldName), null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != COL)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return (Collection<T>)doReadCollection(true, true, null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection(String fieldName,
+        Class<? extends Collection<T>> colCls) throws IgniteObjectException {
+        return readCollection(fieldId(fieldName), colCls);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+        throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != COL)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return (Collection<T>)doReadCollection(true, true, colCls);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName) throws IgniteObjectException {
+        return (Map<K, V>)readMap(fieldId(fieldName), null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != MAP)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return (Map<K, V>)doReadMap(true, true, null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
+        throws IgniteObjectException {
+        return (Map<K, V>)readMap(fieldId(fieldName), mapCls);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls)
+        throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != MAP)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        return (Map<K, V>)doReadMap(true, true, mapCls);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T readEnum(String fieldName)
+        throws IgniteObjectException {
+        return (T)readEnum(fieldId(fieldName), null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T readEnum() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != ENUM)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        Class cls = doReadClass(true);
+
+        return (T)doReadEnum(true, cls);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray(String fieldName)
+        throws IgniteObjectException {
+        return (T[])readEnumArray(fieldId(fieldName), null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray() throws IgniteObjectException {
+        byte flag = doReadByte(true);
+
+        if (flag == NULL)
+            return null;
+
+        if (flag != ENUM_ARR)
+            throw new IgniteObjectException("Invalid flag value: " + flag);
+
+        Class cls = doReadClass(true);
+
+        return (T[])doReadEnumArray(true, cls);
+    }
+
+    /**
+     * @param fieldName Field name.
+     * @return {@code true} if field is set.
+     */
+    public boolean hasField(String fieldName) {
+        return fieldOffset(fieldId(fieldName)) != -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObjectRawReader rawReader() {
+        return this;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Unmarshalled value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable private Object unmarshal(boolean raw) throws IgniteObjectException {
+        return unmarshal(raw, false);
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Unmarshalled value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable private Object unmarshal(boolean raw, boolean detach) throws IgniteObjectException {
+        int start = raw ? rawOff : off;
+
+        byte flag = doReadByte(raw);
+
+        switch (flag) {
+            case NULL:
+                return null;
+
+            case HANDLE:
+                int handle = start - doReadInt(raw);
+
+                IgniteObject handledPo = rCtx.getPortableByHandle(handle);
+
+                if (handledPo != null)
+                    return handledPo;
+
+                off = handle;
+
+                return unmarshal(false);
+
+            case OBJ:
+                PortableUtils.checkProtocolVersion(doReadByte(raw));
+
+                IgniteObjectEx po;
+
+                if (detach) {
+                    in.position(start + GridPortableMarshaller.TOTAL_LEN_POS);
+
+                    int len = in.readInt();
+
+                    in.position(start);
+
+                    po = new IgniteObjectImpl(ctx, in.readByteArray(len), 0);
+                }
+                else
+                    po = in.offheapPointer() > 0
+                        ? new IgniteObjectOffheapImpl(ctx, in.offheapPointer(), start,
+                        in.remaining() + in.position())
+                        : new IgniteObjectImpl(ctx, in.array(), start);
+
+                rCtx.setPortableHandler(start, po);
+
+                if (raw)
+                    rawOff = start + po.length();
+                else
+                    off = start + po.length();
+
+                return po;
+
+            case BYTE:
+                return doReadByte(raw);
+
+            case SHORT:
+                return doReadShort(raw);
+
+            case INT:
+                return doReadInt(raw);
+
+            case LONG:
+                return doReadLong(raw);
+
+            case FLOAT:
+                return doReadFloat(raw);
+
+            case DOUBLE:
+                return doReadDouble(raw);
+
+            case CHAR:
+                return doReadChar(raw);
+
+            case BOOLEAN:
+                return doReadBoolean(raw);
+
+            case DECIMAL:
+                return doReadDecimal(raw);
+
+            case STRING:
+                return doReadString(raw);
+
+            case UUID:
+                return doReadUuid(raw);
+
+            case DATE:
+                return doReadDate(raw);
+
+            case TIMESTAMP:
+                return doReadTimestamp(raw);
+
+            case BYTE_ARR:
+                return doReadByteArray(raw);
+
+            case SHORT_ARR:
+                return doReadShortArray(raw);
+
+            case INT_ARR:
+                return doReadIntArray(raw);
+
+            case LONG_ARR:
+                return doReadLongArray(raw);
+
+            case FLOAT_ARR:
+                return doReadFloatArray(raw);
+
+            case DOUBLE_ARR:
+                return doReadDoubleArray(raw);
+
+            case CHAR_ARR:
+                return doReadCharArray(raw);
+
+            case BOOLEAN_ARR:
+                return doReadBooleanArray(raw);
+
+            case DECIMAL_ARR:
+                return doReadDecimalArray(raw);
+
+            case STRING_ARR:
+                return doReadStringArray(raw);
+
+            case UUID_ARR:
+                return doReadUuidArray(raw);
+
+            case DATE_ARR:
+                return doReadDateArray(raw);
+
+            case TIMESTAMP_ARR:
+                return doReadTimestampArray(raw);
+
+            case OBJ_ARR:
+                return doReadObjectArray(raw, false);
+
+            case COL:
+                return doReadCollection(raw, false, null);
+
+            case MAP:
+                return doReadMap(raw, false, null);
+
+            case MAP_ENTRY:
+                return doReadMapEntry(raw, false);
+
+            case PORTABLE_OBJ:
+                return doReadPortableObject(raw);
+
+            case ENUM:
+                return doReadEnum(raw, doReadClass(raw));
+
+            case ENUM_ARR:
+                return doReadEnumArray(raw, doReadClass(raw));
+
+            case CLASS:
+                return doReadInt(raw);
+
+            case OPTM_MARSH:
+                int len = doReadInt(true);
+
+                ByteArrayInputStream input = new ByteArrayInputStream(in.array(), in.position(), len);
+
+                Object obj;
+
+                try {
+                    obj = ctx.optimizedMarsh().unmarshal(input, null);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteObjectException("Failed to unmarshal object with optmMarsh marshaller", e);
+                }
+
+                if (raw)
+                    rawOff += len;
+                else
+                    off += len;
+
+                return obj;
+
+            default:
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+        }
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private byte doReadByte(boolean raw) {
+        in.position(raw ? rawOff++ : off++);
+
+        return in.readByte();
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private short doReadShort(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        short val = in.readShort();
+
+        if (raw)
+            rawOff += 2;
+        else
+            off += 2;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private int doReadInt(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        int val = in.readInt();
+
+        if (raw)
+            rawOff += 4;
+        else
+            off += 4;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private long doReadLong(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        long val = in.readLong();
+
+        if (raw)
+            rawOff += 8;
+        else
+            off += 8;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private float doReadFloat(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        float val = in.readFloat();
+
+        if (raw)
+            rawOff += 4;
+        else
+            off += 4;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private double doReadDouble(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        double val = in.readDouble();
+
+        if (raw)
+            rawOff += 8;
+        else
+            off += 8;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private char doReadChar(boolean raw) {
+        in.position(raw ? rawOff : off);
+
+        char val = in.readChar();
+
+        if (raw)
+            rawOff += 2;
+        else
+            off += 2;
+
+        return val;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private boolean doReadBoolean(boolean raw) {
+        in.position(raw ? rawOff++ : off++);
+
+        return in.readBoolean();
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private BigDecimal doReadDecimal(boolean raw) {
+        int scale = doReadInt(raw);
+        byte[] mag = doReadByteArray(raw);
+
+        BigInteger intVal = new BigInteger(mag);
+
+        if (scale < 0) {
+            scale &= 0x7FFFFFFF;
+
+            intVal = intVal.negate();
+        }
+
+        return new BigDecimal(intVal, scale);
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private String doReadString(boolean raw) {
+        if (doReadBoolean(raw)) {
+            if (!in.hasArray())
+                return new String(doReadByteArray(raw), UTF_8);
+
+            int strLen = doReadInt(raw);
+            int strOff = raw ? rawOff : off;
+
+            String res = new String(in.array(), strOff, strLen, UTF_8);
+
+            if (raw)
+                rawOff += strLen;
+            else
+                off += strLen;
+
+            return res;
+        }
+        else
+            return String.valueOf(doReadCharArray(raw));
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private UUID doReadUuid(boolean raw) {
+        return new UUID(doReadLong(raw), doReadLong(raw));
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private Date doReadDate(boolean raw) {
+        long time = doReadLong(raw);
+
+        return new Date(time);
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private Timestamp doReadTimestamp(boolean raw) {
+        long time = doReadLong(raw);
+        int nanos = doReadInt(raw);
+
+        Timestamp ts = new Timestamp(time);
+
+        ts.setNanos(ts.getNanos() + nanos);
+
+        return ts;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable private Object doReadObject(boolean raw) throws IgniteObjectException {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, in, raw ? rawOff : off, ldr, rCtx);
+
+        Object obj = reader.deserialize();
+
+        if (raw)
+            rawOff += reader.len;
+        else
+            off += reader.len;
+
+        return obj;
+    }
+
+    /**
+     * @return Deserialized object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
+     */
+    @Nullable Object deserialize() throws IgniteObjectException {
+        Object obj;
+
+        byte flag = doReadByte(true);
+
+        switch (flag) {
+            case NULL:
+                obj = null;
+
+                break;
+
+            case HANDLE:
+                int handle = start - doReadInt(true);
+
+                obj = rCtx.getObjectByHandle(handle);
+
+                if (obj == null) {
+                    off = handle;
+
+                    obj = doReadObject(false);
+                }
+
+                break;
+
+            case OBJ:
+                if (typeId == null)
+                    readObjectTypeId(true);
+
+                assert typeId != UNREGISTERED_TYPE_ID;
+
+                PortableUtils.checkProtocolVersion(doReadByte(true));
+
+                boolean userType = doReadBoolean(true);
+
+                // Skip typeId and hash code.
+                rawOff += 8;
+
+                desc = ctx.descriptorForTypeId(userType, typeId, ldr);
+
+                len = doReadInt(true);
+
+                rawOff = start + doReadInt(true);
+
+                if (desc == null)
+                    throw new IgniteObjectInvalidClassException("Unknown type ID: " + typeId);
+
+                obj = desc.read(this);
+
+                break;
+
+            case BYTE:
+                obj = doReadByte(true);
+
+                break;
+
+            case SHORT:
+                obj = doReadShort(true);
+
+                break;
+
+            case INT:
+                obj = doReadInt(true);
+
+                break;
+
+            case LONG:
+                obj = doReadLong(true);
+
+                break;
+
+            case FLOAT:
+                obj = doReadFloat(true);
+
+                break;
+
+            case DOUBLE:
+                obj = doReadDouble(true);
+
+                break;
+
+            case CHAR:
+                obj = doReadChar(true);
+
+                break;
+
+            case BOOLEAN:
+                obj = doReadBoolean(true);
+
+                break;
+
+            case DECIMAL:
+                obj = doReadDecimal(true);
+
+                break;
+
+            case STRING:
+                obj = doReadString(true);
+
+                break;
+
+            case UUID:
+                obj = doReadUuid(true);
+
+                break;
+
+            case DATE:
+                obj = doReadDate(true);
+
+                break;
+
+            case TIMESTAMP:
+                obj = doReadTimestamp(true);
+
+                break;
+
+            case BYTE_ARR:
+                obj = doReadByteArray(true);
+
+                break;
+
+            case SHORT_ARR:
+                obj = doReadShortArray(true);
+
+                break;
+
+            case INT_ARR:
+                obj = doReadIntArray(true);
+
+                break;
+
+            case LONG_ARR:
+                obj = doReadLongArray(true);
+
+                break;
+
+            case FLOAT_ARR:
+                obj = doReadFloatArray(true);
+
+                break;
+
+            case DOUBLE_ARR:
+                obj = doReadDoubleArray(true);
+
+                break;
+
+            case CHAR_ARR:
+                obj = doReadCharArray(true);
+
+                break;
+
+            case BOOLEAN_ARR:
+                obj = doReadBooleanArray(true);
+
+                break;
+
+            case DECIMAL_ARR:
+                obj = doReadDecimalArray(true);
+
+                break;
+
+            case STRING_ARR:
+                obj = doReadStringArray(true);
+
+                break;
+
+            case UUID_ARR:
+                obj = doReadUuidArray(true);
+
+                break;
+
+            case DATE_ARR:
+                obj = doReadDateArray(true);
+
+                break;
+
+            case TIMESTAMP_ARR:
+                obj = doReadTimestampArray(true);
+
+                break;
+
+            case OBJ_ARR:
+                obj = doReadObjectArray(true, true);
+
+                break;
+
+            case COL:
+                obj = doReadCollection(true, true, null);
+
+                break;
+
+            case MAP:
+                obj = doReadMap(true, true, null);
+
+                break;
+
+            case MAP_ENTRY:
+                obj = doReadMapEntry(true, true);
+
+                break;
+
+            case PORTABLE_OBJ:
+                obj = doReadPortableObject(true);
+
+                ((IgniteObjectImpl)obj).context(ctx);
+
+                if (!GridPortableMarshaller.KEEP_PORTABLES.get())
+                    obj = ((IgniteObject)obj).deserialize();
+
+                break;
+
+            case ENUM:
+                obj = doReadEnum(true, doReadClass(true));
+
+                break;
+
+            case ENUM_ARR:
+                obj = doReadEnumArray(true, doReadClass(true));
+
+                break;
+
+            case CLASS:
+                obj = doReadClass(true);
+
+                break;
+
+            case OPTM_MARSH:
+                int len = doReadInt(true);
+
+                ByteArrayInputStream input = new ByteArrayInputStream(in.array(), in.position(), len);
+
+                try {
+                    obj = ctx.optimizedMarsh().unmarshal(input, null);
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteObjectException("Failed to unmarshal object with optimized marshaller", e);
+                }
+
+                rawOff += len;
+
+                break;
+
+            default:
+                throw new IgniteObjectException("Invalid flag value: " + flag);
+        }
+
+        if (len == 0)
+            len = rawOff - start;
+
+        return obj;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private byte[] doReadByteArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        byte[] arr = in.readByteArray(len);
+
+        setHandler(arr, hPos);
+
+        if (raw)
+            rawOff += len;
+        else
+            off += len;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private short[] doReadShortArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        short[] arr = in.readShortArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 1;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private int[] doReadIntArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        int[] arr = in.readIntArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 2;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private long[] doReadLongArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        long[] arr = in.readLongArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 3;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private float[] doReadFloatArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        float[] arr = in.readFloatArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 2;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private double[] doReadDoubleArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        double[] arr = in.readDoubleArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 3;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private char[] doReadCharArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        char[] arr = in.readCharArray(len);
+
+        setHandler(arr, hPos);
+
+        int bytes = len << 1;
+
+        if (raw)
+            rawOff += bytes;
+        else
+            off += bytes;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private boolean[] doReadBooleanArray(boolean raw) {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        in.position(raw ? rawOff : off);
+
+        boolean[] arr = in.readBooleanArray(len);
+
+        setHandler(arr, hPos);
+
+        if (raw)
+            rawOff += len;
+        else
+            off += len;
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private BigDecimal[] doReadDecimalArray(boolean raw) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        BigDecimal[] arr = new BigDecimal[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else {
+                if (flag != DECIMAL)
+                    throw new IgniteObjectException("Invalid flag value: " + flag);
+
+                arr[i] = doReadDecimal(raw);
+            }
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private String[] doReadStringArray(boolean raw) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        String[] arr = new String[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else {
+                if (flag != STRING)
+                    throw new IgniteObjectException("Invalid flag value: " + flag);
+
+                arr[i] = doReadString(raw);
+            }
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private UUID[] doReadUuidArray(boolean raw) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        UUID[] arr = new UUID[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else {
+                if (flag != UUID)
+                    throw new IgniteObjectException("Invalid flag value: " + flag);
+
+                arr[i] = doReadUuid(raw);
+            }
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private Date[] doReadDateArray(boolean raw) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        Date[] arr = new Date[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else {
+                if (flag != DATE)
+                    throw new IgniteObjectException("Invalid flag value: " + flag);
+
+                arr[i] = doReadDate(raw);
+            }
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private Timestamp[] doReadTimestampArray(boolean raw) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int len = doReadInt(raw);
+
+        Timestamp[] arr = new Timestamp[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else {
+                if (flag != TIMESTAMP)
+                    throw new IgniteObjectException("Invalid flag value: " + flag);
+
+                arr[i] = doReadTimestamp(raw);
+            }
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param deep Deep flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private Object[] doReadObjectArray(boolean raw, boolean deep) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        Class compType = doReadClass(raw);
+
+        int len = doReadInt(raw);
+
+        Object[] arr = deep ? (Object[])Array.newInstance(compType, len) : new Object[len];
+
+        setHandler(arr, hPos);
+
+        for (int i = 0; i < len; i++)
+            arr[i] = deep ? doReadObject(raw) : unmarshal(raw);
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param deep Deep flag.
+     * @param cls Collection class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @SuppressWarnings("unchecked")
+    private Collection<?> doReadCollection(boolean raw, boolean deep, @Nullable Class<? extends Collection> cls)
+        throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int size = doReadInt(raw);
+
+        assert size >= 0;
+
+        byte colType = doReadByte(raw);
+
+        Collection<Object> col;
+
+        if (cls != null) {
+            try {
+                Constructor<? extends Collection> cons = cls.getConstructor();
+
+                col = cons.newInstance();
+            }
+            catch (NoSuchMethodException ignored) {
+                throw new IgniteObjectException("Collection class doesn't have public default constructor: " +
+                    cls.getName());
+            }
+            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
+                throw new IgniteObjectException("Failed to instantiate collection: " + cls.getName(), e);
+            }
+        }
+        else {
+            switch (colType) {
+                case ARR_LIST:
+                    col = new ArrayList<>(size);
+
+                    break;
+
+                case LINKED_LIST:
+                    col = new LinkedList<>();
+
+                    break;
+
+                case HASH_SET:
+                    col = U.newHashSet(size);
+
+                    break;
+
+                case LINKED_HASH_SET:
+                    col = U.newLinkedHashSet(size);
+
+                    break;
+
+                case TREE_SET:
+                    col = new TreeSet<>();
+
+                    break;
+
+                case CONC_SKIP_LIST_SET:
+                    col = new ConcurrentSkipListSet<>();
+
+                    break;
+
+                case USER_SET:
+                    col = U.newHashSet(size);
+
+                    break;
+
+                case USER_COL:
+                    col = new ArrayList<>(size);
+
+                    break;
+
+                default:
+                    throw new IgniteObjectException("Invalid collection type: " + colType);
+            }
+        }
+
+        setHandler(col, hPos);
+
+        for (int i = 0; i < size; i++)
+            col.add(deep ? doReadObject(raw) : unmarshal(raw));
+
+        return col;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param deep Deep flag.
+     * @param cls Map class.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @SuppressWarnings("unchecked")
+    private Map<?, ?> doReadMap(boolean raw, boolean deep, @Nullable Class<? extends Map> cls)
+        throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        int size = doReadInt(raw);
+
+        assert size >= 0;
+
+        byte mapType = doReadByte(raw);
+
+        Map<Object, Object> map;
+
+        if (cls != null) {
+            try {
+                Constructor<? extends Map> cons = cls.getConstructor();
+
+                map = cons.newInstance();
+            }
+            catch (NoSuchMethodException ignored) {
+                throw new IgniteObjectException("Map class doesn't have public default constructor: " +
+                    cls.getName());
+            }
+            catch (InvocationTargetException | InstantiationException | IllegalAccessException e) {
+                throw new IgniteObjectException("Failed to instantiate map: " + cls.getName(), e);
+            }
+        }
+        else {
+            switch (mapType) {
+                case HASH_MAP:
+                    map = U.newHashMap(size);
+
+                    break;
+
+                case LINKED_HASH_MAP:
+                    map = U.newLinkedHashMap(size);
+
+                    break;
+
+                case TREE_MAP:
+                    map = new TreeMap<>();
+
+                    break;
+
+                case CONC_HASH_MAP:
+                    map = new ConcurrentHashMap<>(size);
+
+                    break;
+
+                case USER_COL:
+                    map = U.newHashMap(size);
+
+                    break;
+
+                case PROPERTIES_MAP:
+                    map = new Properties();
+
+                    break;
+
+                default:
+                    throw new IgniteObjectException("Invalid map type: " + mapType);
+            }
+        }
+
+        setHandler(map, hPos);
+
+        for (int i = 0; i < size; i++)
+            map.put(deep ? doReadObject(raw) : unmarshal(raw), deep ? doReadObject(raw) : unmarshal(raw));
+
+        return map;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param deep Deep flag.
+     * @return Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    private Map.Entry<?, ?> doReadMapEntry(boolean raw, boolean deep) throws IgniteObjectException {
+        int hPos = (raw ? rawOff : off) - 1;
+
+        Object val1 = deep ? doReadObject(raw) : unmarshal(raw);
+        Object val2 = deep ? doReadObject(raw) : unmarshal(raw);
+
+        GridMapEntry entry = new GridMapEntry<>(val1, val2);
+
+        setHandler(entry, hPos);
+
+        return entry;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private IgniteObject doReadPortableObject(boolean raw) {
+        if (in.offheapPointer() > 0) {
+            int len = doReadInt(raw);
+
+            int pos = raw ? rawOff : off;
+
+            if (raw)
+                rawOff += len;
+            else
+                off += len;
+
+            int start = doReadInt(raw);
+
+            return new IgniteObjectOffheapImpl(ctx, in.offheapPointer() + pos, start, len);
+        }
+        else {
+            byte[] arr = doReadByteArray(raw);
+            int start = doReadInt(raw);
+
+            return new IgniteObjectImpl(ctx, arr, start);
+        }
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param cls Enum class.
+     * @return Value.
+     */
+    private Enum<?> doReadEnum(boolean raw, Class<?> cls) throws IgniteObjectException {
+        if (!cls.isEnum())
+            throw new IgniteObjectException("Class does not represent enum type: " + cls.getName());
+
+        int ord = doReadInt(raw);
+
+        return ord >= 0 ? (Enum<?>)GridEnumCache.get(cls)[ord] : null;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param cls Enum class.
+     * @return Value.
+     */
+    private Object[] doReadEnumArray(boolean raw, Class<?> cls) throws IgniteObjectException {
+        int len = doReadInt(raw);
+
+        Object[] arr = (Object[])Array.newInstance(cls, len);
+
+        for (int i = 0; i < len; i++) {
+            byte flag = doReadByte(raw);
+
+            if (flag == NULL)
+                arr[i] = null;
+            else
+                arr[i] = doReadEnum(raw, doReadClass(raw));
+        }
+
+        return arr;
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @return Value.
+     */
+    private Class doReadClass(boolean raw) throws IgniteObjectException {
+        return doReadClass(raw, doReadInt(raw));
+    }
+
+    /**
+     * @param raw Raw flag.
+     * @param typeId Type id.
+     * @return Value.
+     */
+    private Class doReadClass(boolean raw, int typeId) throws IgniteObjectException {
+        Class cls;
+
+        if (typeId == OBJECT_TYPE_ID)
+            return Object.class;
+
+        if (typeId != UNREGISTERED_TYPE_ID)
+            cls = ctx.descriptorForTypeId(true, typeId, ldr).describedClass();
+        else {
+            byte flag = doReadByte(raw);
+
+            if (flag != STRING)
+                throw new IgniteObjectException("No class definition for typeId: " + typeId);
+
+            String clsName = doReadString(raw);
+
+            try {
+                cls = U.forName(clsName, ldr);
+            }
+            catch (ClassNotFoundException e) {
+                throw new IgniteObjectInvalidClassException("Failed to load the class: " + clsName, e);
+            }
+
+            // forces registering of class by type id, at least locally
+            ctx.descriptorForClass(cls);
+        }
+
+        return cls;
+    }
+
+    /**
+     * @param name Field name.
+     * @return Field offset.
+     */
+    private int fieldId(String name) {
+        assert name != null;
+
+        if (typeId == null)
+            readObjectTypeId(false);
+
+        assert typeId != UNREGISTERED_TYPE_ID;
+
+        return ctx.fieldId(typeId, name);
+    }
+
+    /**
+     * @param id Field ID.
+     * @return Field offset.
+     */
+    private int fieldOffset(int id) {
+        assert hdrLen != 0;
+
+        int off = start + hdrLen;
+
+        int end = start + in.readInt(start + RAW_DATA_OFF_POS);
+
+        while (true) {
+            if (off >= end)
+                return -1;
+
+            int id0 = in.readInt(off);
+
+            if (id0 == id)
+                return off + 8;
+
+            int len = in.readInt(off + 4);
+
+            off += (8 + len);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedByte() throws IOException {
+        return readByte() & 0xff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedShort() throws IOException {
+        return readShort() & 0xffff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String readLine() throws IOException {
+        SB sb = new SB();
+
+        int b;
+
+        while ((b = read()) >= 0) {
+            char c = (char)b;
+
+            switch (c) {
+                case '\n':
+                    return sb.toString();
+
+                case '\r':
+                    b = read();
+
+                    if (b < 0 || b == '\n')
+                        return sb.toString();
+                    else
+                        sb.a((char)b);
+
+                    break;
+
+                default:
+                    sb.a(c);
+            }
+        }
+
+        return sb.toString();
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public String readUTF() throws IOException {
+        return readString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(byte[] b

<TRUNCATED>

[17/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectEx.java
new file mode 100644
index 0000000..e10b048
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectEx.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;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.IdentityHashMap;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Internal portable object interface.
+ */
+public abstract class IgniteObjectEx implements IgniteObject {
+    /**
+     * @return Length.
+     */
+    public abstract int length();
+
+    /**
+     * @return Object start.
+     */
+    public abstract int start();
+
+    /**
+     * @return {@code True} if object is array based.
+     */
+    protected abstract boolean hasArray();
+
+    /**
+     * @return Object array if object is array based, otherwise {@code null}.
+     */
+    public abstract byte[] array();
+
+    /**
+     * @return Object offheap address is object is offheap based, otherwise 0.
+     */
+    public abstract long offheapAddress();
+
+    /**
+     * @param ctx Reader context.
+     * @param fieldName Field name.
+     * @return Field name.
+     */
+    @Nullable protected abstract <F> F field(PortableReaderContext ctx, String fieldName);
+
+    /** {@inheritDoc} */
+    @Override public IgniteObject clone() throws CloneNotSupportedException {
+        return (IgniteObject)super.clone();
+    }
+
+    /** {@inheritDoc} */
+    public boolean equals(Object other) {
+        if (other == this)
+            return true;
+
+        if (other == null)
+            return false;
+
+        if (!(other instanceof IgniteObjectEx))
+            return false;
+
+        IgniteObjectEx otherPo = (IgniteObjectEx)other;
+
+        if (length() != otherPo.length() || typeId() != otherPo.typeId())
+            return false;
+
+        if (hasArray()) {
+            if (otherPo.hasArray()) {
+                int len = length();
+                int end = start() + len;
+
+                byte[] arr = array();
+                byte[] otherArr = otherPo.array();
+
+                for (int i = start(), j = otherPo.start(); i < end; i++, j++) {
+                    if (arr[i] != otherArr[j])
+                        return false;
+                }
+
+                return true;
+            }
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(otherPo.offheapAddress() + otherPo.start(), array());
+            }
+        }
+        else {
+            assert offheapAddress() > 0;
+
+            if (otherPo.hasArray())
+                return GridUnsafeMemory.compare(offheapAddress() + start(), otherPo.array());
+            else {
+                assert otherPo.offheapAddress() > 0;
+
+                return GridUnsafeMemory.compare(offheapAddress() + start(),
+                    otherPo.offheapAddress() + otherPo.start(),
+                    length());
+            }
+        }
+    }
+
+    /**
+     * @param ctx Reader context.
+     * @param handles Handles for already traversed objects.
+     * @return String representation.
+     */
+    private String toString(PortableReaderContext ctx, IdentityHashMap<IgniteObject, Integer> handles) {
+        int idHash = System.identityHashCode(this);
+
+        IgniteObjectMetadata meta;
+
+        try {
+            meta = metaData();
+        }
+        catch (IgniteObjectException ignore) {
+            meta = null;
+        }
+
+        if (meta == null)
+            return "PortableObject [hash=" + idHash + ", typeId=" + typeId() + ']';
+
+        handles.put(this, idHash);
+
+        SB buf = new SB(meta.typeName());
+
+        if (meta.fields() != null) {
+            buf.a(" [hash=").a(idHash);
+
+            for (String name : meta.fields()) {
+                Object val = field(ctx, name);
+
+                buf.a(", ").a(name).a('=');
+
+                if (val instanceof byte[])
+                    buf.a(Arrays.toString((byte[]) val));
+                else if (val instanceof short[])
+                    buf.a(Arrays.toString((short[])val));
+                else if (val instanceof int[])
+                    buf.a(Arrays.toString((int[])val));
+                else if (val instanceof long[])
+                    buf.a(Arrays.toString((long[])val));
+                else if (val instanceof float[])
+                    buf.a(Arrays.toString((float[])val));
+                else if (val instanceof double[])
+                    buf.a(Arrays.toString((double[])val));
+                else if (val instanceof char[])
+                    buf.a(Arrays.toString((char[])val));
+                else if (val instanceof boolean[])
+                    buf.a(Arrays.toString((boolean[]) val));
+                else if (val instanceof BigDecimal[])
+                    buf.a(Arrays.toString((BigDecimal[])val));
+                else {
+                    if (val instanceof IgniteObjectEx) {
+                        IgniteObjectEx po = (IgniteObjectEx)val;
+
+                        Integer idHash0 = handles.get(val);
+
+                        if (idHash0 != null) {  // Circular reference.
+                            IgniteObjectMetadata meta0 = po.metaData();
+
+                            assert meta0 != null;
+
+                            buf.a(meta0.typeName()).a(" [hash=").a(idHash0).a(", ...]");
+                        }
+                        else
+                            buf.a(po.toString(ctx, handles));
+                    }
+                    else
+                        buf.a(val);
+                }
+            }
+
+            buf.a(']');
+        }
+
+        return buf.toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        try {
+            PortableReaderContext ctx = new PortableReaderContext();
+
+            ctx.setPortableHandler(start(), this);
+
+            return toString(ctx, new IdentityHashMap<IgniteObject, Integer>());
+        }
+        catch (IgniteObjectException e) {
+            throw new IgniteException("Failed to create string representation of portable object.", e);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectImpl.java
new file mode 100644
index 0000000..412ae19
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectImpl.java
@@ -0,0 +1,411 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.IgniteCodeGeneratingFail;
+import org.apache.ignite.internal.portable.streams.PortableHeapInputStream;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable object implementation.
+ */
+@IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator.
+public final class IgniteObjectImpl extends IgniteObjectEx implements Externalizable,
+    Message, CacheObject, KeyCacheObject {
+    /** */
+    public static final byte TYPE_PORTABLE = 100;
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final PortablePrimitives PRIM = PortablePrimitives.get();
+
+    /** */
+    @GridDirectTransient
+    private PortableContext ctx;
+
+    /** */
+    private byte[] arr;
+
+    /** */
+    private int start;
+
+    /** */
+    @GridDirectTransient
+    private Object obj;
+
+    /** */
+    @GridDirectTransient
+    private boolean detachAllowed;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public IgniteObjectImpl() {
+        // No-op.
+    }
+
+    /**
+     * @param ctx Context.
+     * @param arr Array.
+     * @param start Start.
+     */
+    public IgniteObjectImpl(PortableContext ctx, byte[] arr, int start) {
+        assert ctx != null;
+        assert arr != null;
+
+        this.ctx = ctx;
+        this.arr = arr;
+        this.start = start;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte type() {
+        return TYPE_PORTABLE;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPlatformType() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean internal() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        Object obj0 = obj;
+
+        if (obj0 == null || cpy)
+            obj0 = deserializeValue();
+
+        return (T)obj0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
+        if (detached())
+            return array();
+
+        int len = length();
+
+        byte[] arr0 = new byte[len];
+
+        U.arrayCopy(arr, start, arr0, 0, len);
+
+        return arr0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        if (detached())
+            return this;
+
+        return (IgniteObjectImpl)detach();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        this.ctx = ((CacheObjectPortableProcessorImpl)ctx.processor()).portableContext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int length() {
+        return PRIM.readInt(arr, start + GridPortableMarshaller.TOTAL_LEN_POS);
+    }
+
+    /**
+     * @return Detached portable object.
+     */
+    public IgniteObject detach() {
+        if (!detachAllowed || detached())
+            return this;
+
+        int len = length();
+
+        byte[] arr0 = new byte[len];
+
+        U.arrayCopy(arr, start, arr0, 0, len);
+
+        return new IgniteObjectImpl(ctx, arr0, 0);
+    }
+
+    /**
+     * @return Detached or not.
+     */
+    public boolean detached() {
+        return start == 0 && length() == arr.length;
+    }
+
+    /**
+     * @return {@code True} if detach is allowed.
+     */
+    public boolean detachAllowed() {
+        return true;
+    }
+
+    /**
+     * @param detachAllowed Detach allowed flag.
+     */
+    public void detachAllowed(boolean detachAllowed) {
+        this.detachAllowed = detachAllowed;
+    }
+
+    /**
+     * @return Context.
+     */
+    public PortableContext context() {
+        return ctx;
+    }
+
+    /**
+     * @param ctx Context.
+     */
+    public void context(PortableContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] array() {
+        return arr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int start() {
+        return start;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long offheapAddress() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasArray() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int typeId() {
+        return PRIM.readInt(arr, start + GridPortableMarshaller.TYPE_ID_POS);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteObjectMetadata metaData() throws IgniteObjectException {
+        if (ctx == null)
+            throw new IgniteObjectException("PortableContext is not set for the object.");
+
+        return ctx.metaData(typeId());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <F> F field(String fieldName) throws IgniteObjectException {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, arr, start, null);
+
+        return (F)reader.unmarshal(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx,
+            new PortableHeapInputStream(arr),
+            start,
+            null,
+            rCtx);
+
+        return (F)reader.unmarshal(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasField(String fieldName) {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, arr, start, null);
+
+        return reader.hasField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T deserialize() throws IgniteObjectException {
+        Object obj0 = obj;
+
+        if (obj0 == null)
+            obj0 = deserializeValue();
+
+        return (T)obj0;
+
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObject clone() throws CloneNotSupportedException {
+        return super.clone();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return PRIM.readInt(arr, start + GridPortableMarshaller.HASH_CODE_POS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(ctx);
+
+        if (detachAllowed) {
+            int len = length();
+
+            out.writeInt(len);
+            out.write(arr, start, len);
+            out.writeInt(0);
+        }
+        else {
+            out.writeInt(arr.length);
+            out.write(arr);
+            out.writeInt(start);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        ctx = (PortableContext)in.readObject();
+
+        arr = new byte[in.readInt()];
+
+        in.readFully(arr);
+
+        start = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByteArray("arr",
+                    arr,
+                    detachAllowed ? start : 0,
+                    detachAllowed ? length() : arr.length))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeInt("start", detachAllowed ? 0 : start))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                arr = reader.readByteArray("arr");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                start = reader.readInt("start");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 113;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+
+    /**
+     * Runs value deserialization regardless of whether obj already has the deserialized value.
+     * Will set obj if descriptor is configured to keep deserialized values.
+     */
+    private Object deserializeValue() {
+        // TODO: IGNITE-1272 - Deserialize with proper class loader.
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, arr, start, null);
+
+        Object obj0 = reader.deserialize();
+
+        PortableClassDescriptor desc = reader.descriptor();
+
+        assert desc != null;
+
+        if (desc.keepDeserialized())
+            obj = obj0;
+
+        return obj0;
+    }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectMetaDataImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectMetaDataImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectMetaDataImpl.java
new file mode 100644
index 0000000..b28fccb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectMetaDataImpl.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectRawReader;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable meta data implementation.
+ */
+public class IgniteObjectMetaDataImpl implements IgniteObjectMetadata, IgniteObjectMarshalAware, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private String typeName;
+
+    /** */
+    @GridToStringInclude
+    private Map<String, String> fields;
+
+    /** */
+    private volatile Map<Integer, String> fldIdToName;
+
+    /** */
+    private String affKeyFieldName;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public IgniteObjectMetaDataImpl() {
+        // No-op.
+    }
+
+    /**
+     * @param typeName Type name.
+     * @param fields Fields map.
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public IgniteObjectMetaDataImpl(String typeName, @Nullable Map<String, String> fields,
+        @Nullable String affKeyFieldName) {
+        assert typeName != null;
+
+        this.typeName = typeName;
+        this.fields = fields;
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String typeName() {
+        return typeName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> fields() {
+        return fields != null ? fields.keySet() : Collections.<String>emptyList();
+    }
+
+    /**
+     * @return Fields.
+     */
+    public Map<String, String> fields0() {
+        return fields != null ? fields : Collections.<String, String>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String fieldTypeName(String fieldName) {
+        return fields != null ? fields.get(fieldName) : null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String affinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * @return Fields meta data.
+     */
+    public Map<String, String> fieldsMeta() {
+        return fields != null ? fields : Collections.<String, String>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, typeName);
+        U.writeMap(out, fields);
+        U.writeString(out, affKeyFieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeName = U.readString(in);
+        fields = U.readMap(in);
+        affKeyFieldName = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+        IgniteObjectRawWriter raw = writer.rawWriter();
+
+        raw.writeString(typeName);
+        raw.writeString(affKeyFieldName);
+        raw.writeMap(fields);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+        IgniteObjectRawReader raw = reader.rawReader();
+
+        typeName = raw.readString();
+        affKeyFieldName = raw.readString();
+        fields = raw.readMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteObjectMetaDataImpl.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectOffheapImpl.java
new file mode 100644
index 0000000..02d4b04
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectOffheapImpl.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+/**
+ *  Portable object implementation over offheap memory
+ */
+public class IgniteObjectOffheapImpl extends IgniteObjectEx implements Externalizable, CacheObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final long ptr;
+
+    /** */
+    private final int start;
+
+    /** */
+    private final int size;
+
+    /**
+     * For {@link Externalizable} (not supported).
+     */
+    public IgniteObjectOffheapImpl() {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * @param ctx Context.
+     * @param ptr Memory address.
+     * @param start Object start.
+     * @param size Memory size.
+     */
+    public IgniteObjectOffheapImpl(PortableContext ctx, long ptr, int start, int size) {
+        this.ctx = ctx;
+        this.ptr = ptr;
+        this.start = start;
+        this.size = size;
+    }
+
+    /**
+     * @return Heap-based copy.
+     */
+    public IgniteObject heapCopy() {
+        return new IgniteObjectImpl(ctx, U.copyMemory(ptr, size), start);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int typeId() {
+        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TYPE_ID_POS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int length() {
+        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TOTAL_LEN_POS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.HASH_CODE_POS);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int start() {
+        return start;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] array() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long offheapAddress() {
+        return ptr;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean hasArray() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteObjectMetadata metaData() throws IgniteObjectException {
+        if (ctx == null)
+            throw new IgniteObjectException("PortableContext is not set for the object.");
+
+        return ctx.metaData(typeId());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <F> F field(String fieldName) throws IgniteObjectException {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx,
+            new PortableOffheapInputStream(ptr, size, false),
+            start,
+            null);
+
+        return (F)reader.unmarshal(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override protected <F> F field(PortableReaderContext rCtx, String fieldName) {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx,
+            new PortableOffheapInputStream(ptr, size, false),
+            start,
+            null,
+            rCtx);
+
+        return (F)reader.unmarshal(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasField(String fieldName) {
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx,
+            new PortableOffheapInputStream(ptr, size, false),
+            start,
+            null);
+
+        return reader.hasField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T deserialize() throws IgniteObjectException {
+        return (T)deserializeValue();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("CloneDoesntCallSuperClone")
+    @Override public IgniteObject clone() throws CloneNotSupportedException {
+        return heapCopy();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte type() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPlatformType() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        return (T)deserializeValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        throw new UnsupportedOperationException(); // To make sure it is not marshalled.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        throw new UnsupportedOperationException(); // To make sure it is not marshalled.
+    }
+
+    /**
+     * @return Deserialized value.
+     */
+    private Object deserializeValue() {
+        // TODO: IGNITE-1272 - Deserialize with proper class loader.
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(
+            ctx,
+            new PortableOffheapInputStream(ptr, size, false),
+            start,
+            null);
+
+        return reader.deserialize();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawReaderEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawReaderEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawReaderEx.java
new file mode 100644
index 0000000..8157178
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawReaderEx.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectRawReader;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Extended reader interface.
+ */
+public interface IgniteObjectRawReaderEx extends IgniteObjectRawReader {
+    /**
+     * @return Object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public Object readObjectDetached() throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawWriterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawWriterEx.java
new file mode 100644
index 0000000..112607c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectRawWriterEx.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.internal.portable.streams.PortableOutputStream;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Extended writer interface.
+ */
+public interface IgniteObjectRawWriterEx extends IgniteObjectRawWriter, AutoCloseable {
+    /**
+     * @param obj Object to write.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public void writeObjectDetached(@Nullable Object obj) throws IgniteObjectException;
+
+    /**
+     * @return Output stream.
+     */
+    public PortableOutputStream out();
+
+    /**
+     * Cleans resources.
+     */
+    @Override public void close();
+
+    /**
+     * Reserve a room for an integer.
+     *
+     * @return Position in the stream where value is to be written.
+     */
+    public int reserveInt();
+
+    /**
+     * Write int value at the specific position.
+     *
+     * @param pos Position.
+     * @param val Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
+     */
+    public void writeInt(int pos, int val) throws IgniteObjectException;
+}
\ No newline at end of file


[18/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 3d38ed9..a91aa7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -31,6 +31,7 @@ import javax.net.ssl.SSLContext;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.store.CacheStoreSessionListener;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
@@ -427,6 +428,9 @@ public class IgniteConfiguration {
     /** Platform configuration. */
     private PlatformConfiguration platformCfg;
 
+    /** Cache key configuration. */
+    private CacheKeyConfiguration[] cacheKeyCfg;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -463,6 +467,7 @@ public class IgniteConfiguration {
         atomicCfg = cfg.getAtomicConfiguration();
         daemon = cfg.isDaemon();
         cacheCfg = cfg.getCacheConfiguration();
+        cacheKeyCfg = cfg.getCacheKeyConfiguration();
         cacheSanityCheckEnabled = cfg.isCacheSanityCheckEnabled();
         connectorCfg = cfg.getConnectorConfiguration();
         classLdr = cfg.getClassLoader();
@@ -1950,6 +1955,25 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Gets cache key configuration.
+     *
+     * @return Cache key configuration.
+     */
+    public CacheKeyConfiguration[] getCacheKeyConfiguration() {
+        return cacheKeyCfg;
+    }
+
+    /**
+     * Sets cache key configuration.
+     * Cache key configuration defines
+     *
+     * @param cacheKeyCfg Cache key configuration.
+     */
+    public void setCacheKeyCfg(CacheKeyConfiguration... cacheKeyCfg) {
+        this.cacheKeyCfg = cacheKeyCfg;
+    }
+
+    /**
      * Gets flag indicating whether cache sanity check is enabled. If enabled, then Ignite
      * will perform the following checks and throw an exception if check fails:
      * <ul>

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObject.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObject.java
new file mode 100644
index 0000000..c3ba1c1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObject.java
@@ -0,0 +1,153 @@
+/*
+ * 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.igniteobject;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Wrapper for portable object in portable binary format. Once an object is defined as portable,
+ * Ignite will always store it in memory in the portable (i.e. binary) format.
+ * User can choose to work either with the portable format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * <b>NOTE:</b> user does not need to (and should not) implement this interface directly.
+ * <p>
+ * To work with the portable format directly, user should create a cache projection
+ * over {@code PortableObject} class and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepBinary();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use GridPortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in the portable objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the portable binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the portable
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side portable object will have the fields A, B, and C.
+ * As the structure of a portable object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Building Portable Objects</h1>
+ * Ignite comes with {@link IgniteObjectBuilder} which allows to build portable objects dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to portable format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * </pre>
+ * <h1 class="header">Portable Metadata</h1>
+ * Even though Ignite portable protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all portable types which
+ * can be queried ar runtime via any of the {@link org.apache.ignite.IgniteObjects#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code PortableObject.toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface IgniteObject extends Serializable, Cloneable {
+    /**
+     * Gets portable object type ID.
+     *
+     * @return Type ID.
+     */
+    public int typeId();
+
+    /**
+     * Gets meta data for this portable object.
+     *
+     * @return Meta data.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metaData() throws IgniteObjectException;
+
+    /**
+     * Gets field value.
+     *
+     * @param fieldName Field name.
+     * @return Field value.
+     * @throws IgniteObjectException In case of any other error.
+     */
+    @Nullable public <F> F field(String fieldName) throws IgniteObjectException;
+
+    /**
+     * Checks whether field is set.
+     *
+     * @param fieldName Field name.
+     * @return {@code true} if field is set.
+     */
+    public boolean hasField(String fieldName);
+
+    /**
+     * Gets fully deserialized instance of portable object.
+     *
+     * @return Fully deserialized instance of portable object.
+     * @throws IgniteObjectInvalidClassException If class doesn't exist.
+     * @throws IgniteObjectException In case of any other error.
+     */
+    @Nullable public <T> T deserialize() throws IgniteObjectException;
+
+    /**
+     * Copies this portable object.
+     *
+     * @return Copy of this portable object.
+     */
+    public IgniteObject clone() throws CloneNotSupportedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectBuilder.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectBuilder.java
new file mode 100644
index 0000000..08e1c2a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectBuilder.java
@@ -0,0 +1,136 @@
+/*
+ * 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.igniteobject;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable object builder. Provides ability to build portable objects dynamically without having class definitions.
+ * <p>
+ * Here is an example of how a portable object can be built dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ *
+ * <p>
+ * Also builder can be initialized by existing portable object. This allows changing some fields without affecting
+ * other fields.
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder(person);
+ *
+ * builder.setField("name", "John");
+ *
+ * person = builder.build();
+ * </pre>
+ * </p>
+ *
+ * If you need to modify nested portable object you can get builder for nested object using
+ * {@link #getField(String)}, changes made on nested builder will affect parent object,
+ * for example:
+ *
+ * <pre name=code class=java>
+ * PortableBuilder personBuilder = grid.portables().createBuilder(personPortableObj);
+ * PortableBuilder addressBuilder = personBuilder.setField("address");
+ *
+ * addressBuilder.setField("city", "New York");
+ *
+ * personPortableObj = personBuilder.build();
+ *
+ * // Should be "New York".
+ * String city = personPortableObj.getField("address").getField("city");
+ * </pre>
+ *
+ * @see org.apache.ignite.IgniteObjects#builder(int)
+ * @see org.apache.ignite.IgniteObjects#builder(String)
+ * @see org.apache.ignite.IgniteObjects#builder(IgniteObject)
+ */
+public interface IgniteObjectBuilder {
+    /**
+     * Returns value assigned to the specified field.
+     * If the value is a portable object instance of {@code GridPortableBuilder} will be returned,
+     * which can be modified.
+     * <p>
+     * Collections and maps returned from this method are modifiable.
+     *
+     * @param name Field name.
+     * @return Filed value.
+     */
+    public <T> T getField(String name);
+
+    /**
+     * Sets field value.
+     *
+     * @param name Field name.
+     * @param val Field value (cannot be {@code null}).
+     * @see IgniteObject#metaData()
+     */
+    public IgniteObjectBuilder setField(String name, Object val);
+
+    /**
+     * Sets field value with value type specification.
+     * <p>
+     * Field type is needed for proper metadata update.
+     *
+     * @param name Field name.
+     * @param val Field value.
+     * @param type Field type.
+     * @see IgniteObject#metaData()
+     */
+    public <T> IgniteObjectBuilder setField(String name, @Nullable T val, Class<? super T> type);
+
+    /**
+     * Sets field value.
+     * <p>
+     * This method should be used if field is portable object.
+     *
+     * @param name Field name.
+     * @param builder Builder for object field.
+     */
+    public IgniteObjectBuilder setField(String name, @Nullable IgniteObjectBuilder builder);
+
+    /**
+     * Removes field from this builder.
+     *
+     * @param fieldName Field name.
+     * @return {@code this} instance for chaining.
+     */
+    public IgniteObjectBuilder removeField(String fieldName);
+
+    /**
+     * Sets hash code for resulting portable object returned by {@link #build()} method.
+     * <p>
+     * If not set {@code 0} is used.
+     *
+     * @param hashCode Hash code.
+     * @return {@code this} instance for chaining.
+     */
+    public IgniteObjectBuilder hashCode(int hashCode);
+
+    /**
+     * Builds portable object.
+     *
+     * @return Portable object.
+     * @throws IgniteObjectException In case of error.
+     */
+    public IgniteObject build() throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectConfiguration.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectConfiguration.java
new file mode 100644
index 0000000..20a4dec
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectConfiguration.java
@@ -0,0 +1,155 @@
+/*
+ * 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.igniteobject;
+
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Defines configuration properties for a specific portable type. Providing per-type
+ * configuration is optional, as it is generally enough, and also optional, to provide global portable
+ * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
+ * However, this class allows you to change configuration properties for a specific
+ * portable type without affecting configuration for other portable types.
+ * <p>
+ * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
+ */
+public class IgniteObjectConfiguration {
+    /** Class name. */
+    private String clsName;
+
+    /** ID mapper. */
+    private IgniteObjectIdMapper idMapper;
+
+    /** Serializer. */
+    private IgniteObjectSerializer serializer;
+
+    /** Meta data enabled flag. */
+    private Boolean metaDataEnabled;
+
+    /** Keep deserialized flag. */
+    private Boolean keepDeserialized;
+
+    /**
+     */
+    public IgniteObjectConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public IgniteObjectConfiguration(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets type name.
+     *
+     * @return Type name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param clsName Type name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public IgniteObjectIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(IgniteObjectIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public IgniteObjectSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(IgniteObjectSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Defines whether meta data is collected for this type. If provided, this value will override
+     * {@link PortableMarshaller#isMetaDataEnabled()} property.
+     *
+     * @return Whether meta data is collected.
+     */
+    public Boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(Boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * Defines whether {@link IgniteObject} should cache deserialized instance. If provided,
+     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
+     * property.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteObjectConfiguration.class, this, super.toString());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectException.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectException.java
new file mode 100644
index 0000000..c86c17f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectException.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igniteobject;
+
+import org.apache.ignite.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating portable object serialization error.
+ */
+public class IgniteObjectException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates portable exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteObjectException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates portable exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public IgniteObjectException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates portable exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public IgniteObjectException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectIdMapper.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectIdMapper.java
new file mode 100644
index 0000000..ea11824
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectIdMapper.java
@@ -0,0 +1,56 @@
+/*
+ * 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.igniteobject;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Type and field ID mapper for portable objects. Ignite never writes full
+ * strings for field or type names. Instead, for performance reasons, Ignite
+ * writes integer hash codes for type and field names. It has been tested that
+ * hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide {@code PortableIdMapper} allows to override the automatically
+ * generated hash code IDs for the type and field names.
+ * <p>
+ * Portable ID mapper can be configured for all portable objects via {@link PortableMarshaller#getIdMapper()} method,
+ * or for a specific portable type via {@link IgniteObjectConfiguration#getIdMapper()} method.
+ */
+public interface IgniteObjectIdMapper {
+    /**
+     * Gets type ID for provided class name.
+     * <p>
+     * If {@code 0} is returned, hash code of class simple name will be used.
+     *
+     * @param clsName Class name.
+     * @return Type ID.
+     */
+    public int typeId(String clsName);
+
+    /**
+     * Gets ID for provided field.
+     * <p>
+     * If {@code 0} is returned, hash code of field name will be used.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectInvalidClassException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectInvalidClassException.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectInvalidClassException.java
new file mode 100644
index 0000000..8b2b223
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectInvalidClassException.java
@@ -0,0 +1,58 @@
+/*
+ * 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.igniteobject;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating that class needed for deserialization of portable object does not exist.
+ * <p>
+ * Thrown from {@link IgniteObject#deserialize()} method.
+ */
+public class IgniteObjectInvalidClassException extends IgniteObjectException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates invalid class exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public IgniteObjectInvalidClassException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates invalid class exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public IgniteObjectInvalidClassException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates invalid class exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public IgniteObjectInvalidClassException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMarshalAware.java
new file mode 100644
index 0000000..6670431
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMarshalAware.java
@@ -0,0 +1,48 @@
+/*
+ * 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.igniteobject;
+
+/**
+ * Interface that allows to implement custom serialization
+ * logic for portable objects. IgniteObject is not required
+ * to implement this interface, in which case Ignite will automatically
+ * serialize portable objects using reflection.
+ * <p>
+ * This interface, in a way, is analogous to {@link java.io.Externalizable}
+ * interface, which allows users to override default serialization logic,
+ * usually for performance reasons. The only difference here is that portable
+ * serialization is already very fast and implementing custom serialization
+ * logic for portables does not provide significant performance gains.
+ */
+public interface IgniteObjectMarshalAware {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param writer Portable object writer.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param reader Portable object reader.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void readPortable(IgniteObjectReader reader) throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMetadata.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMetadata.java
new file mode 100644
index 0000000..943d774
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectMetadata.java
@@ -0,0 +1,60 @@
+/*
+ * 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.igniteobject;
+
+import java.util.Collection;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable type meta data. Metadata for portable types can be accessed from any of the
+ * {@link org.apache.ignite.IgniteObjects#metadata(String)} methods.
+ * Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface IgniteObjectMetadata {
+    /**
+     * Gets portable type name.
+     *
+     * @return Portable type name.
+     */
+    public String typeName();
+
+    /**
+     * Gets collection of all field names for this portable type.
+     *
+     * @return Collection of all field names for this portable type.
+     */
+    public Collection<String> fields();
+
+    /**
+     * Gets name of the field type for a given field.
+     *
+     * @param fieldName Field name.
+     * @return Field type name.
+     */
+    @Nullable public String fieldTypeName(String fieldName);
+
+    /**
+     * Portable objects can optionally specify custom key-affinity mapping in the
+     * configuration. This method returns the name of the field which should be
+     * used for the key-affinity mapping.
+     *
+     * @return Affinity key field name.
+     */
+    @Nullable public String affinityKeyFieldName();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawReader.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawReader.java
new file mode 100644
index 0000000..e908900
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawReader.java
@@ -0,0 +1,240 @@
+/*
+ * 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.igniteobject;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw reader for portable objects. Raw reader does not use field name hash codes, therefore,
+ * making the format even more compact. However, if the raw reader is used,
+ * dynamic structure changes to the portable objects are not supported.
+ */
+public interface IgniteObjectRawReader {
+    /**
+     * @return Byte value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public byte readByte() throws IgniteObjectException;
+
+    /**
+     * @return Short value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public short readShort() throws IgniteObjectException;
+
+    /**
+     * @return Integer value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public int readInt() throws IgniteObjectException;
+
+    /**
+     * @return Long value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public long readLong() throws IgniteObjectException;
+
+    /**
+     * @return Float value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public float readFloat() throws IgniteObjectException;
+
+    /**
+     * @return Double value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public double readDouble() throws IgniteObjectException;
+
+    /**
+     * @return Char value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public char readChar() throws IgniteObjectException;
+
+    /**
+     * @return Boolean value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public boolean readBoolean() throws IgniteObjectException;
+
+    /**
+     * @return Decimal value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public BigDecimal readDecimal() throws IgniteObjectException;
+
+    /**
+     * @return String value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public String readString() throws IgniteObjectException;
+
+    /**
+     * @return UUID.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public UUID readUuid() throws IgniteObjectException;
+
+    /**
+     * @return Date.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Date readDate() throws IgniteObjectException;
+
+    /**
+     * @return Timestamp.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Timestamp readTimestamp() throws IgniteObjectException;
+
+    /**
+     * @return Object.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> T readObject() throws IgniteObjectException;
+
+    /**
+     * @return Byte array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public byte[] readByteArray() throws IgniteObjectException;
+
+    /**
+     * @return Short array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public short[] readShortArray() throws IgniteObjectException;
+
+    /**
+     * @return Integer array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public int[] readIntArray() throws IgniteObjectException;
+
+    /**
+     * @return Long array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public long[] readLongArray() throws IgniteObjectException;
+
+    /**
+     * @return Float array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public float[] readFloatArray() throws IgniteObjectException;
+
+    /**
+     * @return Byte array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public double[] readDoubleArray() throws IgniteObjectException;
+
+    /**
+     * @return Char array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public char[] readCharArray() throws IgniteObjectException;
+
+    /**
+     * @return Boolean array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray() throws IgniteObjectException;
+
+    /**
+     * @return Decimal array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public BigDecimal[] readDecimalArray() throws IgniteObjectException;
+
+    /**
+     * @return String array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public String[] readStringArray() throws IgniteObjectException;
+
+    /**
+     * @return UUID array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public UUID[] readUuidArray() throws IgniteObjectException;
+
+    /**
+     * @return Date array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Date[] readDateArray() throws IgniteObjectException;
+
+    /**
+     * @return Timestamp array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Timestamp[] readTimestampArray() throws IgniteObjectException;
+
+    /**
+     * @return Object array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Object[] readObjectArray() throws IgniteObjectException;
+
+    /**
+     * @return Collection.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection() throws IgniteObjectException;
+
+    /**
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+        throws IgniteObjectException;
+
+    /**
+     * @return Map.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap() throws IgniteObjectException;
+
+    /**
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls) throws IgniteObjectException;
+
+    /**
+     * @return Value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum() throws IgniteObjectException;
+
+    /**
+     * @return Value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray() throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawWriter.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawWriter.java
new file mode 100644
index 0000000..a1bfd83
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectRawWriter.java
@@ -0,0 +1,225 @@
+/*
+ * 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.igniteobject;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw writer for portable object. Raw writer does not write field name hash codes, therefore,
+ * making the format even more compact. However, if the raw writer is used,
+ * dynamic structure changes to the portable objects are not supported.
+ */
+public interface IgniteObjectRawWriter {
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeByte(byte val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeShort(short val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeInt(int val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeLong(long val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeFloat(float val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDouble(double val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeChar(char val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeBoolean(boolean val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDecimal(@Nullable BigDecimal val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeString(@Nullable String val) throws IgniteObjectException;
+
+    /**
+     * @param val UUID to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeUuid(@Nullable UUID val) throws IgniteObjectException;
+
+    /**
+     * @param val Date to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDate(@Nullable Date val) throws IgniteObjectException;
+
+    /**
+     * @param val Timestamp to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeTimestamp(@Nullable Timestamp val) throws IgniteObjectException;
+
+    /**
+     * @param obj Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeObject(@Nullable Object obj) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeByteArray(@Nullable byte[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeShortArray(@Nullable short[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeIntArray(@Nullable int[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeLongArray(@Nullable long[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeFloatArray(@Nullable float[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDoubleArray(@Nullable double[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeCharArray(@Nullable char[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeBooleanArray(@Nullable boolean[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDecimalArray(@Nullable BigDecimal[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeStringArray(@Nullable String[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeUuidArray(@Nullable UUID[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDateArray(@Nullable Date[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeTimestampArray(@Nullable Timestamp[] val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeObjectArray(@Nullable Object[] val) throws IgniteObjectException;
+
+    /**
+     * @param col Collection to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T> void writeCollection(@Nullable Collection<T> col) throws IgniteObjectException;
+
+    /**
+     * @param map Map to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <K, V> void writeMap(@Nullable Map<K, V> map) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(T val) throws IgniteObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(T[] val) throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectReader.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectReader.java
new file mode 100644
index 0000000..bad4473
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectReader.java
@@ -0,0 +1,291 @@
+/*
+ * 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.igniteobject;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Reader for portable objects used in {@link IgniteObjectMarshalAware} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link IgniteObjectIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface IgniteObjectReader {
+    /**
+     * @param fieldName Field name.
+     * @return Byte value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public byte readByte(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public short readShort(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public int readInt(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public long readLong(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @throws IgniteObjectException In case of error.
+     * @return Float value.
+     */
+    public float readFloat(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Double value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public double readDouble(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public char readChar(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean value.
+     * @throws IgniteObjectException In case of error.
+     */
+    public boolean readBoolean(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public BigDecimal readDecimal(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public String readString(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public UUID readUuid(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Date readDate(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Timestamp.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Timestamp readTimestamp(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> T readObject(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public byte[] readByteArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public short[] readShortArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public int[] readIntArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public long[] readLongArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Float array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public float[] readFloatArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public double[] readDoubleArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public char[] readCharArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public BigDecimal[] readDecimalArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public String[] readStringArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public UUID[] readUuidArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Date[] readDateArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Timestamp array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Timestamp[] readTimestampArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object array.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public Object[] readObjectArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Collection.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
+        throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Map.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
+        throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IgniteObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws IgniteObjectException;
+
+    /**
+     * Gets raw reader. Raw reader does not use field name hash codes, therefore,
+     * making the format even more compact. However, if the raw reader is used,
+     * dynamic structure changes to the portable objects are not supported.
+     *
+     * @return Raw reader.
+     */
+    public IgniteObjectRawReader rawReader();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectSerializer.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectSerializer.java
new file mode 100644
index 0000000..2ac3d0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectSerializer.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.igniteobject;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Interface that allows to implement custom serialization logic for portable objects.
+ * Can be used instead of {@link IgniteObjectMarshalAware} in case if the class
+ * cannot be changed directly.
+ * <p>
+ * Portable serializer can be configured for all portable objects via
+ * {@link PortableMarshaller#getSerializer()} method, or for a specific
+ * portable type via {@link IgniteObjectConfiguration#getSerializer()} method.
+ */
+public interface IgniteObjectSerializer {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param obj Empty object.
+     * @param writer Portable object writer.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writePortable(Object obj, IgniteObjectWriter writer) throws IgniteObjectException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param obj Empty object
+     * @param reader Portable object reader.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void readPortable(Object obj, IgniteObjectReader reader) throws IgniteObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectWriter.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectWriter.java
new file mode 100644
index 0000000..3409f07
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/IgniteObjectWriter.java
@@ -0,0 +1,273 @@
+/*
+ * 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.igniteobject;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Writer for portable object used in {@link IgniteObjectMarshalAware} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link IgniteObjectIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface IgniteObjectWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val UUID to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeUuid(String fieldName, @Nullable UUID val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Date to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDate(String fieldName, @Nullable Date val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Timestamp to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeDateArray(String fieldName, @Nullable Date[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IgniteObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IgniteObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IgniteObjectException;
+
+    /**
+     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
+     * making the format even more compact. However, if the raw writer is used,
+     * dynamic structure changes to the portable objects are not supported.
+     *
+     * @return Raw writer.
+     */
+    public IgniteObjectRawWriter rawWriter();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/igniteobject/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igniteobject/package-info.java b/modules/core/src/main/java/org/apache/ignite/igniteobject/package-info.java
new file mode 100644
index 0000000..abc305e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igniteobject/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 portable objects API classes.
+ */
+package org.apache.ignite.igniteobject;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 6cddb47..f2e98c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -32,7 +32,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -125,7 +125,6 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
@@ -2765,7 +2764,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() {
+    @Override public IgniteObjects portables() {
         return ((CacheObjectPortableProcessor)ctx.cacheObjects()).portables();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 079015c..61fa93a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -31,7 +31,7 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentRequest;
 import org.apache.ignite.internal.managers.deployment.GridDeploymentResponse;
 import org.apache.ignite.internal.managers.eventstorage.GridEventStorageMessage;
-import org.apache.ignite.internal.portable.PortableObjectImpl;
+import org.apache.ignite.internal.portable.IgniteObjectImpl;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryInfoCollection;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicateContainsValue;
@@ -680,7 +680,7 @@ public class GridIoMessageFactory implements MessageFactory {
                 break;
 
             case 113:
-                msg = new PortableObjectImpl();
+                msg = new IgniteObjectImpl();
 
                 break;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index 6f16755..2bd121c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.igniteobject.IgniteObjectException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -230,13 +230,13 @@ public class GridPortableMarshaller {
      * @param obj Object to marshal.
      * @param off Offset.
      * @return Byte array.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public byte[] marshal(@Nullable Object obj, int off) throws PortableException {
+    public byte[] marshal(@Nullable Object obj, int off) throws IgniteObjectException {
         if (obj == null)
             return new byte[] { NULL };
 
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, off)) {
+        try (IgniteObjectWriterExImpl writer = new IgniteObjectWriterExImpl(ctx, off)) {
             writer.marshal(obj, false);
 
             return writer.array();
@@ -246,13 +246,13 @@ public class GridPortableMarshaller {
     /**
      * @param bytes Bytes array.
      * @return Portable object.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     @SuppressWarnings("unchecked")
-    @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws PortableException {
+    @Nullable public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteObjectException {
         assert bytes != null;
 
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, bytes, 0, clsLdr);
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, bytes, 0, clsLdr);
 
         return (T)reader.unmarshal();
     }
@@ -260,10 +260,10 @@ public class GridPortableMarshaller {
     /**
      * @param in Input stream.
      * @return Portable object.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     @SuppressWarnings("unchecked")
-    @Nullable public <T> T unmarshal(PortableInputStream in) throws PortableException {
+    @Nullable public <T> T unmarshal(PortableInputStream in) throws IgniteObjectException {
         return (T)reader(in).unmarshal();
     }
 
@@ -271,17 +271,17 @@ public class GridPortableMarshaller {
      * @param arr Byte array.
      * @param ldr Class loader.
      * @return Deserialized object.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     @SuppressWarnings("unchecked")
-    @Nullable public <T> T deserialize(byte[] arr, @Nullable ClassLoader ldr) throws PortableException {
+    @Nullable public <T> T deserialize(byte[] arr, @Nullable ClassLoader ldr) throws IgniteObjectException {
         assert arr != null;
         assert arr.length > 0;
 
         if (arr[0] == NULL)
             return null;
 
-        PortableReaderExImpl reader = new PortableReaderExImpl(ctx, arr, 0, ldr);
+        IgniteObjectReaderExImpl reader = new IgniteObjectReaderExImpl(ctx, arr, 0, ldr);
 
         return (T)reader.deserialize();
     }
@@ -292,8 +292,8 @@ public class GridPortableMarshaller {
      * @param out Output stream.
      * @return Writer.
      */
-    public PortableWriterExImpl writer(PortableOutputStream out) {
-        return new PortableWriterExImpl(ctx, out, 0);
+    public IgniteObjectWriterExImpl writer(PortableOutputStream out) {
+        return new IgniteObjectWriterExImpl(ctx, out, 0);
     }
 
     /**
@@ -302,9 +302,9 @@ public class GridPortableMarshaller {
      * @param in Input stream.
      * @return Reader.
      */
-    public PortableReaderExImpl reader(PortableInputStream in) {
+    public IgniteObjectReaderExImpl reader(PortableInputStream in) {
         // TODO: IGNITE-1272 - Is class loader needed here?
-        return new PortableReaderExImpl(ctx, in, in.position(), null);
+        return new IgniteObjectReaderExImpl(ctx, in, in.position(), null);
     }
 
     /**


[19/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
ignite-950-new WIP


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

Branch: refs/heads/ignite-950-new
Commit: 35b6d61fad1963dfdc6fcfc4fac748acf7974830
Parents: e33fa63
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Oct 30 13:44:08 2015 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Oct 30 13:44:10 2015 +0300

----------------------------------------------------------------------
 .../ignite/examples/portable/Address.java       |   16 +-
 ...mputeClientPortableTaskExecutionExample.java |    4 +-
 .../portable/computegrid/ComputeClientTask.java |   16 +-
 .../CacheClientPortablePutGetExample.java       |   12 +-
 .../CacheClientPortableQueryExample.java        |   30 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    6 +-
 .../java/org/apache/ignite/IgniteCache.java     |    4 +-
 .../java/org/apache/ignite/IgniteObjects.java   |  366 ++
 .../java/org/apache/ignite/IgnitePortables.java |  370 --
 .../ignite/cache/CacheKeyConfiguration.java     |   92 +
 .../org/apache/ignite/cache/QueryEntity.java    |   26 +-
 .../apache/ignite/cache/QueryEntityIndex.java   |  100 -
 .../org/apache/ignite/cache/QueryIndex.java     |  173 +
 .../org/apache/ignite/cache/QueryIndexType.java |   24 +
 .../configuration/CacheConfiguration.java       |   37 +-
 .../configuration/IgniteConfiguration.java      |   24 +
 .../ignite/igniteobject/IgniteObject.java       |  153 +
 .../igniteobject/IgniteObjectBuilder.java       |  136 +
 .../igniteobject/IgniteObjectConfiguration.java |  155 +
 .../igniteobject/IgniteObjectException.java     |   57 +
 .../igniteobject/IgniteObjectIdMapper.java      |   56 +
 .../IgniteObjectInvalidClassException.java      |   58 +
 .../igniteobject/IgniteObjectMarshalAware.java  |   48 +
 .../igniteobject/IgniteObjectMetadata.java      |   60 +
 .../igniteobject/IgniteObjectRawReader.java     |  240 ++
 .../igniteobject/IgniteObjectRawWriter.java     |  225 ++
 .../ignite/igniteobject/IgniteObjectReader.java |  291 ++
 .../igniteobject/IgniteObjectSerializer.java    |   49 +
 .../ignite/igniteobject/IgniteObjectWriter.java |  273 ++
 .../ignite/igniteobject/package-info.java       |   22 +
 .../apache/ignite/internal/IgniteKernal.java    |    5 +-
 .../communication/GridIoMessageFactory.java     |    4 +-
 .../portable/GridPortableMarshaller.java        |   32 +-
 .../internal/portable/IgniteObjectEx.java       |  214 ++
 .../internal/portable/IgniteObjectImpl.java     |  411 +++
 .../portable/IgniteObjectMetaDataCollector.java |  263 ++
 .../portable/IgniteObjectMetaDataImpl.java      |  150 +
 .../portable/IgniteObjectOffheapImpl.java       |  255 ++
 .../portable/IgniteObjectRawReaderEx.java       |   33 +
 .../portable/IgniteObjectRawWriterEx.java       |   60 +
 .../portable/IgniteObjectReaderExImpl.java      | 3230 ++++++++++++++++++
 .../portable/IgniteObjectWriterExImpl.java      | 1892 ++++++++++
 .../portable/PortableClassDescriptor.java       |   88 +-
 .../internal/portable/PortableContext.java      |  147 +-
 .../portable/PortableMetaDataCollector.java     |  263 --
 .../portable/PortableMetaDataHandler.java       |   12 +-
 .../internal/portable/PortableMetaDataImpl.java |  150 -
 .../internal/portable/PortableObjectEx.java     |  214 --
 .../internal/portable/PortableObjectImpl.java   |  411 ---
 .../portable/PortableObjectOffheapImpl.java     |  255 --
 .../internal/portable/PortableRawReaderEx.java  |   33 -
 .../internal/portable/PortableRawWriterEx.java  |   60 -
 .../portable/PortableReaderContext.java         |    8 +-
 .../internal/portable/PortableReaderExImpl.java | 3230 ------------------
 .../ignite/internal/portable/PortableUtils.java |   10 +-
 .../internal/portable/PortableWriterExImpl.java | 1892 ----------
 .../builder/IgniteObjectBuilderImpl.java        |  540 +++
 .../portable/builder/PortableBuilderEnum.java   |    8 +-
 .../portable/builder/PortableBuilderImpl.java   |  544 ---
 .../portable/builder/PortableBuilderReader.java |   56 +-
 .../PortableBuilderSerializationAware.java      |    2 +-
 .../builder/PortableBuilderSerializer.java      |   29 +-
 .../builder/PortableEnumArrayLazyValue.java     |   12 +-
 .../portable/builder/PortableLazyArrayList.java |    2 +-
 .../builder/PortableLazyLinkedList.java         |    2 +-
 .../portable/builder/PortableLazyMap.java       |    2 +-
 .../portable/builder/PortableLazyMapEntry.java  |    2 +-
 .../portable/builder/PortableLazySet.java       |    4 +-
 .../builder/PortableModifiableLazyValue.java    |    2 +-
 .../builder/PortableObjectArrayLazyValue.java   |    8 +-
 .../builder/PortablePlainLazyValue.java         |    2 +-
 .../builder/PortablePlainPortableObject.java    |   22 +-
 .../portable/builder/PortableValueWithType.java |    4 +-
 .../streams/PortableAbstractInputStream.java    |    6 +-
 .../internal/processors/cache/CacheObject.java  |    2 +-
 .../processors/cache/IgniteCacheProxy.java      |    2 +-
 .../CacheDefaultPortableAffinityKeyMapper.java  |    6 +-
 .../portable/CacheObjectPortableContext.java    |    8 -
 .../portable/CacheObjectPortableProcessor.java  |   24 +-
 .../CacheObjectPortableProcessorImpl.java       |  202 +-
 .../cache/portable/IgniteObjectsImpl.java       |  177 +
 .../cache/portable/IgnitePortablesImpl.java     |  177 -
 .../platform/PlatformAbstractTarget.java        |   30 +-
 .../processors/platform/PlatformContext.java    |   26 +-
 .../platform/PlatformContextImpl.java           |   47 +-
 .../platform/PlatformExtendedException.java     |    4 +-
 .../platform/PlatformProcessorImpl.java         |    4 +-
 .../platform/cache/PlatformCache.java           |   40 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    6 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   12 +-
 .../platform/cache/PlatformCacheIterator.java   |    4 +-
 .../PlatformCachePartialUpdateException.java    |    4 +-
 .../cache/affinity/PlatformAffinity.java        |    8 +-
 .../query/PlatformAbstractQueryCursor.java      |   10 +-
 .../PlatformContinuousQueryRemoteFilter.java    |    4 +-
 .../cache/query/PlatformFieldsQueryCursor.java  |    4 +-
 .../cache/query/PlatformQueryCursor.java        |    4 +-
 .../cache/store/PlatformCacheStoreCallback.java |    6 +-
 .../platform/cluster/PlatformClusterGroup.java  |   12 +-
 .../cluster/PlatformClusterNodeFilterImpl.java  |    4 +-
 .../platform/compute/PlatformAbstractJob.java   |    4 +-
 .../platform/compute/PlatformAbstractTask.java  |    6 +-
 .../platform/compute/PlatformClosureJob.java    |    4 +-
 .../platform/compute/PlatformCompute.java       |   24 +-
 .../platform/compute/PlatformFullJob.java       |    6 +-
 .../platform/compute/PlatformFullTask.java      |   12 +-
 .../datastreamer/PlatformDataStreamer.java      |    4 +-
 .../PlatformStreamReceiverImpl.java             |    4 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   48 +-
 .../PlatformDotNetConfigurationClosure.java     |   14 +-
 .../events/PlatformEventFilterListenerImpl.java |    6 +-
 .../platform/events/PlatformEvents.java         |   18 +-
 .../messaging/PlatformMessageFilterImpl.java    |    6 +-
 .../messaging/PlatformMessageLocalFilter.java   |    4 +-
 .../platform/messaging/PlatformMessaging.java   |    8 +-
 .../services/PlatformAbstractService.java       |   16 +-
 .../platform/services/PlatformServices.java     |   20 +-
 .../transactions/PlatformTransactions.java      |    5 +-
 .../platform/utils/PlatformFutureUtils.java     |   10 +-
 .../platform/utils/PlatformReaderBiClosure.java |    4 +-
 .../platform/utils/PlatformReaderClosure.java   |    4 +-
 .../platform/utils/PlatformUtils.java           |   60 +-
 .../platform/utils/PlatformWriterBiClosure.java |    4 +-
 .../platform/utils/PlatformWriterClosure.java   |    4 +-
 .../processors/query/GridQueryProcessor.java    |   28 +-
 .../marshaller/portable/PortableMarshaller.java |   42 +-
 .../apache/ignite/portable/PortableBuilder.java |  137 -
 .../ignite/portable/PortableException.java      |   57 -
 .../ignite/portable/PortableIdMapper.java       |   56 -
 .../portable/PortableInvalidClassException.java |   58 -
 .../ignite/portable/PortableMarshalAware.java   |   48 -
 .../ignite/portable/PortableMetadata.java       |   61 -
 .../apache/ignite/portable/PortableObject.java  |  154 -
 .../apache/ignite/portable/PortableReader.java  |  291 --
 .../ignite/portable/PortableSerializer.java     |   49 -
 .../portable/PortableTypeConfiguration.java     |  177 -
 .../apache/ignite/portable/PortableWriter.java  |  273 --
 .../apache/ignite/portable/package-info.java    |   22 -
 ...idIgniteObjectBuilderAdditionalSelfTest.java | 1289 +++++++
 .../GridIgniteObjectBuilderSelfTest.java        | 1069 ++++++
 ...tBuilderStringAsCharsAdditionalSelfTest.java |   28 +
 ...gniteObjectBuilderStringAsCharsSelfTest.java |   28 +
 .../GridPortableAffinityKeySelfTest.java        |   10 +-
 .../GridPortableBuilderAdditionalSelfTest.java  | 1289 -------
 .../portable/GridPortableBuilderSelfTest.java   | 1069 ------
 ...eBuilderStringAsCharsAdditionalSelfTest.java |   28 -
 ...ridPortableBuilderStringAsCharsSelfTest.java |   28 -
 ...idPortableMarshallerCtxDisabledSelfTest.java |   24 +-
 .../GridPortableMarshallerSelfTest.java         |  444 +--
 .../GridPortableMetaDataDisabledSelfTest.java   |   40 +-
 .../portable/GridPortableMetaDataSelfTest.java  |   40 +-
 .../portable/GridPortableWildcardsSelfTest.java |   70 +-
 ...GridIgniteObjectMarshalerAwareTestClass.java |   67 +
 .../GridPortableMarshalerAwareTestClass.java    |   67 -
 .../mutabletest/GridPortableTestClasses.java    |    6 +-
 ...ntNodeIgniteObjectMetadataMultinodeTest.java |  295 ++
 ...CacheClientNodeIgniteObjectMetadataTest.java |  290 ++
 ...ClientNodePortableMetadataMultinodeTest.java |  295 --
 ...GridCacheClientNodePortableMetadataTest.java |  286 --
 ...niteObjectsAbstractDataStreamerSelfTest.java |  190 ++
 ...iteObjectsAbstractMultiThreadedSelfTest.java |  231 ++
 .../GridCacheIgniteObjectsAbstractSelfTest.java |  981 ++++++
 ...ableObjectsAbstractDataStreamerSelfTest.java |  190 --
 ...bleObjectsAbstractMultiThreadedSelfTest.java |  231 --
 ...ridCachePortableObjectsAbstractSelfTest.java |  981 ------
 ...GridCachePortableStorePortablesSelfTest.java |    6 +-
 ...ridPortableCacheEntryMemorySizeSelfTest.java |    8 +-
 ...leDuplicateIndexObjectsAbstractSelfTest.java |    8 +-
 .../DataStreamProcessorPortableSelfTest.java    |    6 +-
 .../GridDataStreamerImplSelfTest.java           |   26 +-
 ...ridCacheAffinityRoutingPortableSelfTest.java |   10 +-
 ...rtableDataStreamerMultithreadedSelfTest.java |    4 +-
 ...tionedOnlyPortableMultithreadedSelfTest.java |    4 +-
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 +
 ...IgniteObjectsAtomicNearDisabledSelfTest.java |   51 +
 ...gniteObjectsAtomicOffheapTieredSelfTest.java |   29 +
 .../GridCacheIgniteObjectsAtomicSelfTest.java   |   51 +
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 +
 ...eObjectsPartitionedNearDisabledSelfTest.java |   51 +
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 +
 ...idCacheIgniteObjectsPartitionedSelfTest.java |   51 +
 ...eapTieredEvictionAtomicPortableSelfTest.java |    6 +-
 ...heOffHeapTieredEvictionPortableSelfTest.java |    6 +-
 ...AtomicNearDisabledOffheapTieredSelfTest.java |   29 -
 ...rtableObjectsAtomicNearDisabledSelfTest.java |   51 -
 ...tableObjectsAtomicOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsAtomicSelfTest.java |   51 -
 ...tionedNearDisabledOffheapTieredSelfTest.java |   30 -
 ...eObjectsPartitionedNearDisabledSelfTest.java |   51 -
 ...ObjectsPartitionedOffheapTieredSelfTest.java |   30 -
 ...CachePortableObjectsPartitionedSelfTest.java |   51 -
 ...ridCacheIgniteObjectsReplicatedSelfTest.java |   51 +
 ...dCachePortableObjectsReplicatedSelfTest.java |   51 -
 ...idCacheIgniteObjectsAtomicLocalSelfTest.java |   32 +
 ...IgniteObjectsLocalOffheapTieredSelfTest.java |   29 +
 .../GridCacheIgniteObjectsLocalSelfTest.java    |   51 +
 ...CachePortableObjectsAtomicLocalSelfTest.java |   32 -
 ...rtableObjectsLocalOffheapTieredSelfTest.java |   29 -
 .../GridCachePortableObjectsLocalSelfTest.java  |   51 -
 .../PlatformComputePortableArgTask.java         |    8 +-
 .../platform/PlatformEventsWriteEventTask.java  |    6 +-
 .../ignite/testframework/junits/IgniteMock.java |    4 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    4 +-
 .../IgnitePortableObjectsTestSuite.java         |   72 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    2 +-
 206 files changed, 15821 insertions(+), 15164 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
index cb08b25..1c23e41 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/Address.java
@@ -17,19 +17,19 @@
 
 package org.apache.ignite.examples.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 
 /**
  * Employee address.
  * <p>
- * This class implements {@link PortableMarshalAware} only for example purposes,
+ * This class implements {@link org.apache.ignite.igniteobject.IgniteObjectMarshalAware} only for example purposes,
  * in order to show how to customize serialization and deserialization of
  * portable objects.
  */
-public class Address implements PortableMarshalAware {
+public class Address implements IgniteObjectMarshalAware {
     /** Street. */
     private String street;
 
@@ -53,13 +53,13 @@ public class Address implements PortableMarshalAware {
     }
 
     /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
+    @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
         writer.writeString("street", street);
         writer.writeInt("zip", zip);
     }
 
     /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
+    @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
         street = reader.readString("street");
         zip = reader.readInt("zip");
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
index 34d9cde..15eed07 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientPortableTaskExecutionExample.java
@@ -25,7 +25,7 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.examples.portable.Address;
 import org.apache.ignite.examples.portable.Employee;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with task execution.
@@ -74,7 +74,7 @@ public class ComputeClientPortableTaskExecutionExample {
             // Convert collection of employees to collection of portable objects.
             // This allows to send objects across nodes without requiring to have
             // Employee class on classpath of these nodes.
-            Collection<PortableObject> portables = ignite.portables().toPortable(employees);
+            Collection<IgniteObject> portables = ignite.portables().toPortable(employees);
 
             // Execute task and get average salary.
             Long avgSalary = ignite.compute(ignite.cluster().forRemotes()).execute(new ComputeClientTask(), portables);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
index 0eee8c6..94b4e0c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/computegrid/ComputeClientTask.java
@@ -25,7 +25,7 @@ import org.apache.ignite.compute.ComputeJobAdapter;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeTaskSplitAdapter;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -37,19 +37,19 @@ import org.jetbrains.annotations.Nullable;
  * for each batch. After all jobs are executed, there results are reduced to
  * get the average salary.
  */
-public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<PortableObject>, Long> {
+public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<IgniteObject>, Long> {
     /** {@inheritDoc} */
     @Override protected Collection<? extends ComputeJob> split(
         int gridSize,
-        Collection<PortableObject> arg
+        Collection<IgniteObject> arg
     ) {
         Collection<ComputeClientJob> jobs = new ArrayList<>();
 
-        Collection<PortableObject> employees = new ArrayList<>();
+        Collection<IgniteObject> employees = new ArrayList<>();
 
         // Split provided collection into batches and
         // create a job for each batch.
-        for (PortableObject employee : arg) {
+        for (IgniteObject employee : arg) {
             employees.add(employee);
 
             if (employees.size() == 3) {
@@ -85,12 +85,12 @@ public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<Portab
      */
     private static class ComputeClientJob extends ComputeJobAdapter {
         /** Collection of employees. */
-        private final Collection<PortableObject> employees;
+        private final Collection<IgniteObject> employees;
 
         /**
          * @param employees Collection of employees.
          */
-        private ComputeClientJob(Collection<PortableObject> employees) {
+        private ComputeClientJob(Collection<IgniteObject> employees) {
             this.employees = employees;
         }
 
@@ -99,7 +99,7 @@ public class ComputeClientTask extends ComputeTaskSplitAdapter<Collection<Portab
             long sum = 0;
             int cnt = 0;
 
-            for (PortableObject employee : employees) {
+            for (IgniteObject employee : employees) {
                 System.out.println(">>> Processing employee: " + employee.field("name"));
 
                 // Get salary from portable object. Note that object

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
index 77c5d95..793bfbf 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortablePutGetExample.java
@@ -32,7 +32,7 @@ import org.apache.ignite.examples.portable.Address;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
 import org.apache.ignite.examples.portable.Organization;
 import org.apache.ignite.examples.portable.OrganizationType;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with Ignite cache.
@@ -133,10 +133,10 @@ public class CacheClientPortablePutGetExample {
         cache.put(1, org);
 
         // Get cache that will get values as portable objects.
-        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+        IgniteCache<Integer, IgniteObject> portableCache = cache.withKeepBinary();
 
         // Get recently created organization as a portable object.
-        PortableObject po = portableCache.get(1);
+        IgniteObject po = portableCache.get(1);
 
         // Get organization's name from portable object (note that
         // object doesn't need to be fully deserialized).
@@ -212,16 +212,16 @@ public class CacheClientPortablePutGetExample {
         cache.putAll(map);
 
         // Get cache that will get values as portable objects.
-        IgniteCache<Integer, PortableObject> portableCache = cache.withKeepPortable();
+        IgniteCache<Integer, IgniteObject> portableCache = cache.withKeepBinary();
 
         // Get recently created organizations as portable objects.
-        Map<Integer, PortableObject> poMap = portableCache.getAll(map.keySet());
+        Map<Integer, IgniteObject> poMap = portableCache.getAll(map.keySet());
 
         Collection<String> names = new ArrayList<>();
 
         // Get organizations' names from portable objects (note that
         // objects don't need to be fully deserialized).
-        for (PortableObject po : poMap.values())
+        for (IgniteObject po : poMap.values())
             names.add(po.<String>field("name"));
 
         System.out.println();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
index 3170864..cbeed12 100644
--- a/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/portable/datagrid/CacheClientPortableQueryExample.java
@@ -39,7 +39,7 @@ import org.apache.ignite.examples.portable.EmployeeKey;
 import org.apache.ignite.examples.portable.ExamplePortableNodeStartup;
 import org.apache.ignite.examples.portable.Organization;
 import org.apache.ignite.examples.portable.OrganizationType;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * This example demonstrates use of portable objects with cache queries.
@@ -98,7 +98,7 @@ public class CacheClientPortableQueryExample {
                 populateCache(orgCache, employeeCache);
 
                 // Get cache that will work with portable objects.
-                IgniteCache<PortableObject, PortableObject> portableCache = employeeCache.withKeepPortable();
+                IgniteCache<IgniteObject, IgniteObject> portableCache = employeeCache.withKeepBinary();
 
                 // Run SQL query example.
                 sqlQuery(portableCache);
@@ -180,17 +180,17 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class, "zip = ?");
+    private static void sqlQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        SqlQuery<IgniteObject, IgniteObject> query = new SqlQuery<>(Employee.class, "zip = ?");
 
         int zip = 94109;
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query.setArgs(zip));
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees = cache.query(query.setArgs(zip));
 
         System.out.println();
         System.out.println(">>> Employees with zip " + zip + ':');
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue().deserialize());
     }
 
@@ -199,20 +199,20 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlJoinQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        SqlQuery<PortableObject, PortableObject> query = new SqlQuery<>(Employee.class,
+    private static void sqlJoinQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        SqlQuery<IgniteObject, IgniteObject> query = new SqlQuery<>(Employee.class,
             "from Employee, \"" + ORGANIZATION_CACHE_NAME + "\".Organization as org " +
                 "where Employee.organizationId = org._key and org.name = ?");
 
         String organizationName = "GridGain";
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees =
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees =
             cache.query(query.setArgs(organizationName));
 
         System.out.println();
         System.out.println(">>> Employees working for " + organizationName + ':');
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue());
     }
 
@@ -221,7 +221,7 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void sqlFieldsQuery(IgniteCache<PortableObject, PortableObject> cache) {
+    private static void sqlFieldsQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
         SqlFieldsQuery query = new SqlFieldsQuery("select name, salary from Employee");
 
         QueryCursor<List<?>> employees = cache.query(query);
@@ -238,15 +238,15 @@ public class CacheClientPortableQueryExample {
      *
      * @param cache Ignite cache.
      */
-    private static void textQuery(IgniteCache<PortableObject, PortableObject> cache) {
-        TextQuery<PortableObject, PortableObject> query = new TextQuery<>(Employee.class, "TX");
+    private static void textQuery(IgniteCache<IgniteObject, IgniteObject> cache) {
+        TextQuery<IgniteObject, IgniteObject> query = new TextQuery<>(Employee.class, "TX");
 
-        QueryCursor<Cache.Entry<PortableObject, PortableObject>> employees = cache.query(query);
+        QueryCursor<Cache.Entry<IgniteObject, IgniteObject>> employees = cache.query(query);
 
         System.out.println();
         System.out.println(">>> Employees living in Texas:");
 
-        for (Cache.Entry<PortableObject, PortableObject> e : employees.getAll())
+        for (Cache.Entry<IgniteObject, IgniteObject> e : employees.getAll())
             System.out.println(">>>     " + e.getValue().deserialize());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 62fd020..4125f8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -459,11 +459,11 @@ public interface Ignite extends AutoCloseable {
     public <T extends IgnitePlugin> T plugin(String name) throws PluginNotFoundException;
 
     /**
-     * Gets an instance of {@link IgnitePortables} interface.
+     * Gets an instance of {@link IgniteObjects} interface.
      *
-     * @return Instance of {@link IgnitePortables} interface.
+     * @return Instance of {@link IgniteObjects} interface.
      */
-    public IgnitePortables portables();
+    public IgniteObjects portables();
 
     /**
      * Closes {@code this} instance of grid. This method is identical to calling

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index e0f9f55..d58ba30 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -156,7 +156,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      * (which will be stored in portable format), you should acquire following projection
      * to avoid deserialization:
      * <pre>
-     * IgniteCache<Integer, PortableObject> prj = cache.withKeepPortable();
+     * IgniteCache<Integer, PortableObject> prj = cache.withKeepBinary();
      *
      * // Value is not deserialized and returned in portable format.
      * PortableObject po = prj.get(1);
@@ -167,7 +167,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
      *
      * @return New cache instance for portable objects.
      */
-    public <K1, V1> IgniteCache<K1, V1> withKeepPortable();
+    public <K1, V1> IgniteCache<K1, V1> withKeepBinary();
 
     /**
      * Executes {@link #localLoadCache(IgniteBiPredicate, Object...)} on all cache nodes.

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java b/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
new file mode 100644
index 0000000..4afd15b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteObjects.java
@@ -0,0 +1,366 @@
+/*
+ * 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;
+
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Defines portable objects functionality. With portable objects you are able to:
+ * <ul>
+ * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
+ * <li>Make any object portable with zero code change to your existing code.</li>
+ * <li>Nest portable objects within each other.</li>
+ * <li>Automatically handle {@code circular} or {@code null} references.</li>
+ * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
+ * <li>
+ *      Optionally avoid deserialization of objects on the server side
+ *      (objects are stored in {@link org.apache.ignite.igniteobject.IgniteObject} format).
+ * </li>
+ * <li>Avoid need to have concrete class definitions on the server side.</li>
+ * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
+ * <li>Index into portable objects for querying purposes.</li>
+ * </ul>
+ * <h1 class="header">Working With Portables Directly</h1>
+ * Once an object is defined as portable,
+ * Ignite will always store it in memory in the portable (i.e. binary) format.
+ * User can choose to work either with the portable format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * To work with the portable format directly, user should create a special cache projection
+ * using IgniteCache.withKeepBinary() method and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepBinary();
+ *
+ * // Convert instance of MyKey to portable format.
+ * // We could also use PortableBuilder to create the key in portable format directly.
+ * PortableObject key = grid.portables().toPortable(new MyKey());
+ *
+ * PortableObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * If we used, for example, one of the automatically handled portable types for a key, like integer,
+ * and still wanted to work with binary portable format for values, then we would declare cache projection
+ * as follows:
+ * <pre name=code class=java>
+ * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepBinary();
+ * </pre>
+ * <h1 class="header">Automatic Portable Types</h1>
+ * Note that only portable classes are converted to {@link org.apache.ignite.igniteobject.IgniteObject} format. Following
+ * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
+ * object, and instances of these classes will be stored in cache without changes):
+ * <ul>
+ *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
+ *     <li>Arrays of primitives (byte[], int[], ...)</li>
+ *     <li>{@link String} and array of {@link String}s</li>
+ *     <li>{@link UUID} and array of {@link UUID}s</li>
+ *     <li>{@link Date} and array of {@link Date}s</li>
+ *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
+ *     <li>Enums and array of enums</li>
+ *     <li>
+ *         Maps, collections and array of objects (but objects inside
+ *         them will still be converted if they are portable)
+ *     </li>
+ * </ul>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in the portable objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Building Portable Objects</h1>
+ * Ignite comes with {@link org.apache.ignite.igniteobject.IgniteObjectBuilder} which allows to build portable objects dynamically:
+ * <pre name=code class=java>
+ * PortableBuilder builder = Ignition.ignite().portables().builder();
+ *
+ * builder.typeId("MyObject");
+ *
+ * builder.stringField("fieldA", "A");
+ * build.intField("fieldB", "B");
+ *
+ * PortableObject portableObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to portable format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
+ * </pre>
+ * NOTE: you don't need to convert typed objects to portable format before storing
+ * them in cache, Ignite will do that automatically.
+ * <h1 class="header">Portable Metadata</h1>
+ * Even though Ignite portable protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all portable types which
+ * can be queried ar runtime via any of the {@link IgniteObjects#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
+ * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the portable binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the portable
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side portable object will have the fields A, B, and C.
+ * As the structure of a portable object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Configuration</h1>
+ * By default all your objects are considered as portables and no specific configuration is needed.
+ * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
+ * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
+ * The only requirement Ignite imposes is that your object has an empty
+ * constructor. Note, that since server side does not have to know the class definition,
+ * you only need to list portable objects in configuration on the client side. However, if you
+ * list them on the server side as well, then you get the ability to deserialize portable objects
+ * into concrete types on the server as well as on the client.
+ * <p>
+ * Here is an example of portable configuration (note that star (*) notation is supported):
+ * <pre name=code class=xml>
+ * ...
+ * &lt;!-- Explicit portable objects configuration. --&gt;
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
+ *         &lt;property name="classNames"&gt;
+ *             &lt;list&gt;
+ *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
+ *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * ...
+ * </pre>
+ * or from code:
+ * <pre name=code class=java>
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * PortableMarshaller marsh = new PortableMarshaller();
+ *
+ * marsh.setClassNames(Arrays.asList(
+ *     Employee.class.getName(),
+ *     Address.class.getName())
+ * );
+ *
+ * cfg.setMarshaller(marsh);
+ * </pre>
+ * You can also specify class name for a portable object via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration}.
+ * Do it in case if you need to override other configuration properties on per-type level, like
+ * ID-mapper, or serializer.
+ * <h1 class="header">Custom Affinity Keys</h1>
+ * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
+ * storing objects in cache. For example, if you are caching {@code Employee} object with
+ * {@code Organization}, and want to colocate employees with organization they work for,
+ * so you can process them together, you need to specify an alternate affinity key.
+ * With portable objects you would have to do it as following:
+ * <pre name=code class=xml>
+ * &lt;property name="marshaller"&gt;
+ *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
+ *         ...
+ *         &lt;property name="typeConfigurations"&gt;
+ *             &lt;list&gt;
+ *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
+ *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
+ *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
+ *                 &lt;/bean&gt;
+ *             &lt;/list&gt;
+ *         &lt;/property&gt;
+ *         ...
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * <h1 class="header">Serialization</h1>
+ * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
+ * serialization logic by optionally implementing {@link org.apache.ignite.igniteobject.IgniteObjectMarshalAware} interface, like so:
+ * <pre name=code class=java>
+ * public class Address implements PortableMarshalAware {
+ *     private String street;
+ *     private int zip;
+ *
+ *     // Empty constructor required for portable deserialization.
+ *     public Address() {}
+ *
+ *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
+ *         writer.writeString("street", street);
+ *         writer.writeInt("zip", zip);
+ *     }
+ *
+ *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
+ *         street = reader.readString("street");
+ *         zip = reader.readInt("zip");
+ *     }
+ * }
+ * </pre>
+ * Alternatively, if you cannot change class definitions, you can provide custom serialization
+ * logic in {@link org.apache.ignite.igniteobject.IgniteObjectSerializer} either globally in {@link PortableMarshaller} or
+ * for a specific type via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration} instance.
+ * <p>
+ * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
+ * <ul>
+ *     <li>
+ *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
+ *         It may be used to replace the de-serialized object by another one of your choice.
+ *     </li>
+ *     <li>
+ *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
+ *          allows the developer to provide a replacement object that will be serialized instead of the original one.
+ *     </li>
+ * </ul>
+ *
+ * <h1 class="header">Custom ID Mappers</h1>
+ * Ignite implementation uses name hash codes to generate IDs for class names or field names
+ * internally. However, in cases when you want to provide your own ID mapping schema,
+ * you can provide your own {@link org.apache.ignite.igniteobject.IgniteObjectIdMapper} implementation.
+ * <p>
+ * ID-mapper may be provided either globally in {@link PortableMarshaller},
+ * or for a specific type via {@link org.apache.ignite.igniteobject.IgniteObjectConfiguration} instance.
+ * <h1 class="header">Query Indexing</h1>
+ * Portable objects can be indexed for querying by specifying index fields in
+ * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
+ * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
+ * like so:
+ * <pre name=code class=xml>
+ * ...
+ * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
+ *     ...
+ *     &lt;property name="typeMetadata"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="CacheTypeMetadata"&gt;
+ *                 &lt;property name="type" value="Employee"/&gt;
+ *
+ *                 &lt;!-- Fields to index in ascending order. --&gt;
+ *                 &lt;property name="ascendingFields"&gt;
+ *                     &lt;map&gt;
+ *                     &lt;entry key="name" value="java.lang.String"/&gt;
+ *
+ *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
+ *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
+ *                     &lt;/map&gt;
+ *                 &lt;/property&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ * &lt;/bean&gt;
+ * </pre>
+ */
+public interface IgniteObjects {
+    /**
+     * Gets type ID for given type name.
+     *
+     * @param typeName Type name.
+     * @return Type ID.
+     */
+    public int typeId(String typeName);
+
+    /**
+     * Converts provided object to instance of {@link org.apache.ignite.igniteobject.IgniteObject}.
+     *
+     * @param obj Object to convert.
+     * @return Converted object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public <T> T toPortable(@Nullable Object obj) throws IgniteObjectException;
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeId ID of the type.
+     * @return Newly portable builder.
+     */
+    public IgniteObjectBuilder builder(int typeId);
+
+    /**
+     * Creates new portable builder.
+     *
+     * @param typeName Type name.
+     * @return Newly portable builder.
+     */
+    public IgniteObjectBuilder builder(String typeName);
+
+    /**
+     * Creates portable builder initialized by existing portable object.
+     *
+     * @param portableObj Portable object to initialize builder.
+     * @return Portable builder.
+     */
+    public IgniteObjectBuilder builder(IgniteObject portableObj);
+
+    /**
+     * Gets metadata for provided class.
+     *
+     * @param cls Class.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(Class<?> cls) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for provided class name.
+     *
+     * @param typeName Type name.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(String typeName) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    @Nullable public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException;
+
+    /**
+     * Gets metadata for all known types.
+     *
+     * @return Metadata.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public Collection<IgniteObjectMetadata> metadata() throws IgniteObjectException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
deleted file mode 100644
index 1c63df7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite;
-
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Defines portable objects functionality. With portable objects you are able to:
- * <ul>
- * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
- * <li>Make any object portable with zero code change to your existing code.</li>
- * <li>Nest portable objects within each other.</li>
- * <li>Automatically handle {@code circular} or {@code null} references.</li>
- * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
- * <li>
- *      Optionally avoid deserialization of objects on the server side
- *      (objects are stored in {@link PortableObject} format).
- * </li>
- * <li>Avoid need to have concrete class definitions on the server side.</li>
- * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
- * <li>Index into portable objects for querying purposes.</li>
- * </ul>
- * <h1 class="header">Working With Portables Directly</h1>
- * Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * To work with the portable format directly, user should create a special cache projection
- * using IgniteCache.withKeepPortable() method and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use PortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * If we used, for example, one of the automatically handled portable types for a key, like integer,
- * and still wanted to work with binary portable format for values, then we would declare cache projection
- * as follows:
- * <pre name=code class=java>
- * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepPortable();
- * </pre>
- * <h1 class="header">Automatic Portable Types</h1>
- * Note that only portable classes are converted to {@link PortableObject} format. Following
- * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
- * object, and instances of these classes will be stored in cache without changes):
- * <ul>
- *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
- *     <li>Arrays of primitives (byte[], int[], ...)</li>
- *     <li>{@link String} and array of {@link String}s</li>
- *     <li>{@link UUID} and array of {@link UUID}s</li>
- *     <li>{@link Date} and array of {@link Date}s</li>
- *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
- *     <li>Enums and array of enums</li>
- *     <li>
- *         Maps, collections and array of objects (but objects inside
- *         them will still be converted if they are portable)
- *     </li>
- * </ul>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder();
- *
- * builder.typeId("MyObject");
- *
- * builder.stringField("fieldA", "A");
- * build.intField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * NOTE: you don't need to convert typed objects to portable format before storing
- * them in cache, Ignite will do that automatically.
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Configuration</h1>
- * By default all your objects are considered as portables and no specific configuration is needed.
- * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
- * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
- * The only requirement Ignite imposes is that your object has an empty
- * constructor. Note, that since server side does not have to know the class definition,
- * you only need to list portable objects in configuration on the client side. However, if you
- * list them on the server side as well, then you get the ability to deserialize portable objects
- * into concrete types on the server as well as on the client.
- * <p>
- * Here is an example of portable configuration (note that star (*) notation is supported):
- * <pre name=code class=xml>
- * ...
- * &lt;!-- Explicit portable objects configuration. --&gt;
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *         &lt;property name="classNames"&gt;
- *             &lt;list&gt;
- *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
- *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * ...
- * </pre>
- * or from code:
- * <pre name=code class=java>
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * PortableMarshaller marsh = new PortableMarshaller();
- *
- * marsh.setClassNames(Arrays.asList(
- *     Employee.class.getName(),
- *     Address.class.getName())
- * );
- *
- * cfg.setMarshaller(marsh);
- * </pre>
- * You can also specify class name for a portable object via {@link PortableTypeConfiguration}.
- * Do it in case if you need to override other configuration properties on per-type level, like
- * ID-mapper, or serializer.
- * <h1 class="header">Custom Affinity Keys</h1>
- * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
- * storing objects in cache. For example, if you are caching {@code Employee} object with
- * {@code Organization}, and want to colocate employees with organization they work for,
- * so you can process them together, you need to specify an alternate affinity key.
- * With portable objects you would have to do it as following:
- * <pre name=code class=xml>
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
- *         ...
- *         &lt;property name="typeConfigurations"&gt;
- *             &lt;list&gt;
- *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
- *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
- *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
- *                 &lt;/bean&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *         ...
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <h1 class="header">Serialization</h1>
- * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
- * serialization logic by optionally implementing {@link PortableMarshalAware} interface, like so:
- * <pre name=code class=java>
- * public class Address implements PortableMarshalAware {
- *     private String street;
- *     private int zip;
- *
- *     // Empty constructor required for portable deserialization.
- *     public Address() {}
- *
- *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
- *         writer.writeString("street", street);
- *         writer.writeInt("zip", zip);
- *     }
- *
- *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
- *         street = reader.readString("street");
- *         zip = reader.readInt("zip");
- *     }
- * }
- * </pre>
- * Alternatively, if you cannot change class definitions, you can provide custom serialization
- * logic in {@link PortableSerializer} either globally in {@link PortableMarshaller} or
- * for a specific type via {@link PortableTypeConfiguration} instance.
- * <p>
- * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
- * <ul>
- *     <li>
- *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
- *         It may be used to replace the de-serialized object by another one of your choice.
- *     </li>
- *     <li>
- *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
- *          allows the developer to provide a replacement object that will be serialized instead of the original one.
- *     </li>
- * </ul>
- *
- * <h1 class="header">Custom ID Mappers</h1>
- * Ignite implementation uses name hash codes to generate IDs for class names or field names
- * internally. However, in cases when you want to provide your own ID mapping schema,
- * you can provide your own {@link PortableIdMapper} implementation.
- * <p>
- * ID-mapper may be provided either globally in {@link PortableMarshaller},
- * or for a specific type via {@link PortableTypeConfiguration} instance.
- * <h1 class="header">Query Indexing</h1>
- * Portable objects can be indexed for querying by specifying index fields in
- * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
- * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
- * like so:
- * <pre name=code class=xml>
- * ...
- * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
- *     ...
- *     &lt;property name="typeMetadata"&gt;
- *         &lt;list&gt;
- *             &lt;bean class="CacheTypeMetadata"&gt;
- *                 &lt;property name="type" value="Employee"/&gt;
- *
- *                 &lt;!-- Fields to index in ascending order. --&gt;
- *                 &lt;property name="ascendingFields"&gt;
- *                     &lt;map&gt;
- *                     &lt;entry key="name" value="java.lang.String"/&gt;
- *
- *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
- *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
- *                     &lt;/map&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/list&gt;
- *     &lt;/property&gt;
- * &lt;/bean&gt;
- * </pre>
- */
-public interface IgnitePortables {
-    /**
-     * Gets type ID for given type name.
-     *
-     * @param typeName Type name.
-     * @return Type ID.
-     */
-    public int typeId(String typeName);
-
-    /**
-     * Converts provided object to instance of {@link PortableObject}.
-     *
-     * @param obj Object to convert.
-     * @return Converted object.
-     * @throws PortableException In case of error.
-     */
-    public <T> T toPortable(@Nullable Object obj) throws PortableException;
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeId ID of the type.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(int typeId);
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeName Type name.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(String typeName);
-
-    /**
-     * Creates portable builder initialized by existing portable object.
-     *
-     * @param portableObj Portable object to initialize builder.
-     * @return Portable builder.
-     */
-    public PortableBuilder builder(PortableObject portableObj);
-
-    /**
-     * Gets metadata for provided class.
-     *
-     * @param cls Class.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(Class<?> cls) throws PortableException;
-
-    /**
-     * Gets metadata for provided class name.
-     *
-     * @param typeName Type name.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(String typeName) throws PortableException;
-
-    /**
-     * Gets metadata for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(int typeId) throws PortableException;
-
-    /**
-     * Gets metadata for all known types.
-     *
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    public Collection<PortableMetadata> metadata() throws PortableException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.java
new file mode 100644
index 0000000..39ec2a8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheKeyConfiguration.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.cache;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+public class CacheKeyConfiguration implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Type name. */
+    private String typeName;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     * Creates an empty cache key configuration that should be populated via setters.
+     */
+    public CacheKeyConfiguration() {
+        // Convenience no-op constructor.
+    }
+
+    /**
+     * Creates cache key configuration with given type name and affinity field name.
+     *
+     * @param typeName Type name.
+     * @param affKeyFieldName Affinity field name.
+     */
+    public CacheKeyConfiguration(String typeName, String affKeyFieldName) {
+        this.typeName = typeName;
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /**
+     * Sets type name for which affinity field name is being defined.
+     *
+     * @return Type name.
+     */
+    public String getTypeName() {
+        return typeName;
+    }
+
+    /**
+     * @param typeName Type name for which affinity field name is being defined.
+     */
+    public void setTypeName(String typeName) {
+        this.typeName = typeName;
+    }
+
+    /**
+     * Gets affinity key field name.
+     *
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * Sets affinity key field name.
+     *
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affKeyFieldName) {
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheKeyConfiguration.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index 330f911..cb84c47 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -17,7 +17,6 @@
 package org.apache.ignite.cache;
 
 import java.io.Serializable;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -46,7 +45,7 @@ public class QueryEntity implements Serializable {
     private Map<String, String> aliases = new HashMap<>();
 
     /** Collection of query indexes. */
-    private Map<String, QueryEntityIndex> idxs = new HashMap<>();
+    private Map<String, QueryIndex> idxs = new HashMap<>();
 
     /**
      * Gets key type for this query pair.
@@ -109,7 +108,7 @@ public class QueryEntity implements Serializable {
      *
      * @return Collection of index entities.
      */
-    public Collection<QueryEntityIndex> getIndexes() {
+    public Collection<QueryIndex> getIndexes() {
         return idxs.values();
     }
 
@@ -137,8 +136,8 @@ public class QueryEntity implements Serializable {
      *
      * @param idxs Collection of index entities.
      */
-    public void setIndexes(Collection<QueryEntityIndex> idxs) {
-        for (QueryEntityIndex idx : idxs) {
+    public void setIndexes(Collection<QueryIndex> idxs) {
+        for (QueryIndex idx : idxs) {
             if (!F.isEmpty(idx.getFields())) {
                 if (idx.getName() == null)
                     idx.setName(defaultIndexName(idx));
@@ -170,20 +169,20 @@ public class QueryEntity implements Serializable {
      * @param idxName Index name.
      * @param idxType Index type.
      */
-    public void ensureIndex(String idxName, QueryEntityIndex.Type idxType) {
-        QueryEntityIndex idx = idxs.get(idxName);
+    public void ensureIndex(String idxName, QueryIndexType idxType) {
+        QueryIndex idx = idxs.get(idxName);
 
         if (idx == null) {
-            idx = new QueryEntityIndex();
+            idx = new QueryIndex();
 
             idx.setName(idxName);
-            idx.setType(idxType);
+            idx.setIndexType(idxType);
 
             idxs.put(idxName, idx);
         }
         else
             throw new IllegalArgumentException("An index with the same name and of a different type already exists " +
-                "[idxName=" + idxName + ", existingIdxType=" + idx.getType() + ", newIdxType=" + idxType + ']');
+                "[idxName=" + idxName + ", existingIdxType=" + idx.getIndexType() + ", newIdxType=" + idxType + ']');
     }
 
     /**
@@ -192,13 +191,14 @@ public class QueryEntity implements Serializable {
      * @param idx Index to build name for.
      * @return Index name.
      */
-    public static String defaultIndexName(QueryEntityIndex idx) {
+    public static String defaultIndexName(QueryIndex idx) {
         StringBuilder idxName = new StringBuilder();
 
-        for (String field : idx.getFields()) {
-            idxName.append(field);
+        for (Map.Entry<String, Boolean> field : idx.getFields().entrySet()) {
+            idxName.append(field.getKey());
 
             idxName.append('_');
+            idxName.append(field.getValue() ? "asc_" : "desc_");
         }
 
         for (int i = 0; i < idxName.length(); i++) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
deleted file mode 100644
index 18e7572..0000000
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntityIndex.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.ignite.cache;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-/**
- * Contains list of fields to be indexed. It is possible to provide field name
- * suffixed with index specific extension, for example for {@link Type#SORTED sorted} index
- * the list can be provided as following {@code (id, name asc, age desc)}.
- */
-public class QueryEntityIndex implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Index name. */
-    private String name;
-
-    /** */
-    private Collection<String> fields;
-
-    /** */
-    private Type type;
-
-    /**
-     * Index type.
-     */
-    public enum Type {
-        SORTED, FULLTEXT, GEOSPATIAL
-    }
-
-    /**
-     * Gets index name. Will be automatically set if not provided by a user.
-     *
-     * @return Index name.
-     */
-    public String getName() {
-        return name;
-    }
-
-    /**
-     * Sets index name.
-     *
-     * @param name Index name.
-     */
-    public void setName(String name) {
-        this.name = name;
-    }
-
-    /**
-     * Gets fields included in the index.
-     *
-     * @return Collection of index fields.
-     */
-    public Collection<String> getFields() {
-        return fields;
-    }
-
-    /**
-     * Sets fields included in the index.
-     *
-     * @param fields Collection of index fields.
-     */
-    public void setFields(Collection<String> fields) {
-        this.fields = fields;
-    }
-
-    /**
-     * Gets index type.
-     *
-     * @return Index type.
-     */
-    public Type getType() {
-        return type;
-    }
-
-    /**
-     * Sets index type.
-     *
-     * @param type Index type.
-     */
-    public void setType(Type type) {
-        this.type = type;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
new file mode 100644
index 0000000..4868e9e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndex.java
@@ -0,0 +1,173 @@
+/*
+ * 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.cache;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+
+/**
+ * Contains list of fields to be indexed. It is possible to provide field name
+ * suffixed with index specific extension, for example for {@link QueryIndexType#SORTED sorted} index
+ * the list can be provided as following {@code (id, name asc, age desc)}.
+ */
+@SuppressWarnings("TypeMayBeWeakened")
+public class QueryIndex implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Index name. */
+    private String name;
+
+    /** */
+    private LinkedHashMap<String, Boolean> fields;
+
+    /** */
+    private QueryIndexType type;
+
+    /**
+     * Creates an empty index. Should be populated via setters.
+     */
+    public QueryIndex() {
+        // Empty constructor.
+    }
+
+    /**
+     * Creates index for one field.
+     * If index is sorted, then ascending sorting is used by default.
+     * To specify sort order, use the next method.
+     * This constructor should also have a corresponding setter method.
+     */
+    public QueryIndex(String field, QueryIndexType type) {
+        this(Arrays.asList(field), type);
+    }
+
+    /**
+     * Creates index for one field. The last boolean parameter
+     * is ignored for non-sorted indexes.
+     */
+    public QueryIndex(String field, QueryIndexType type, boolean asc) {
+        fields = new LinkedHashMap<>();
+        fields.put(field, asc);
+
+        this.type = type;
+    }
+
+    /**
+     * Creates index for a collection of fields. If index is sorted, fields will be sorted in
+     * ascending order.
+     *
+     * @param fields Collection of fields to create an index.
+     * @param type Index type.
+     */
+    public QueryIndex(Collection<String> fields, QueryIndexType type) {
+        this.fields = new LinkedHashMap<>();
+
+        for (String field : fields)
+            this.fields.put(field, true);
+
+        this.type = type;
+    }
+
+    /**
+     * Creates index for a collection of fields. The order of fields in the created index will be the same
+     * as iteration order in the passed map. Map value defines whether the index will be ascending.
+     *
+     * @param fields Field name to field sort direction for sorted indexes.
+     * @param type Index type.
+     */
+    public QueryIndex(LinkedHashMap<String, Boolean> fields, QueryIndexType type) {
+        this.fields = fields;
+        this.type = type;
+    }
+
+    /**
+     * Gets index name. Will be automatically set if not provided by a user.
+     *
+     * @return Index name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * Sets index name.
+     *
+     * @param name Index name.
+     */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /**
+     * Gets fields included in the index.
+     *
+     * @return Collection of index fields.
+     */
+    public LinkedHashMap<String, Boolean> getFields() {
+        return fields;
+    }
+
+    /**
+     * Sets fields included in the index.
+     *
+     * @param fields Collection of index fields.
+     */
+    public void setFields(LinkedHashMap<String, Boolean> fields) {
+        this.fields = fields;
+    }
+
+    /**
+     * @return Gets a collection of field names.
+     */
+    public Collection<String> getFieldNames() {
+        return fields.keySet();
+    }
+
+    /**
+     * Sets a collection of field names altogether with the field sorting direction. Sorting direction will be
+     * ignored for non-sorted indexes.
+     *
+     * @param fields Collection of fields.
+     * @param asc Ascending flag.
+     */
+    public void setFieldNames(Collection<String> fields, boolean asc) {
+        this.fields = new LinkedHashMap<>();
+
+        for (String field : fields)
+            this.fields.put(field, asc);
+    }
+
+    /**
+     * Gets index type.
+     *
+     * @return Index type.
+     */
+    public QueryIndexType getIndexType() {
+        return type;
+    }
+
+    /**
+     * Sets index type.
+     *
+     * @param type Index type.
+     */
+    public void setIndexType(QueryIndexType type) {
+        this.type = type;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
new file mode 100644
index 0000000..8df0fa1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryIndexType.java
@@ -0,0 +1,24 @@
+/*
+ * 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.cache;
+
+/**
+ * Index type.
+ */
+public enum QueryIndexType {
+    SORTED, FULLTEXT, GEOSPATIAL
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 3a80b83..91ad609 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -30,7 +30,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
-import java.util.List;
 import java.util.Map;
 import java.util.TreeSet;
 import javax.cache.Cache;
@@ -51,7 +50,8 @@ import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryEntityIndex;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictionFilter;
@@ -72,7 +72,6 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
@@ -1951,9 +1950,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         for (ClassProperty prop : desc.props.values())
             entity.addQueryField(prop.fullName(), U.box(prop.type()).getName(), prop.alias());
 
-        QueryEntityIndex txtIdx = null;
+        QueryIndex txtIdx = null;
 
-        Collection<QueryEntityIndex> idxs = new ArrayList<>();
+        Collection<QueryIndex> idxs = new ArrayList<>();
 
         for (Map.Entry<String, GridQueryIndexDescriptor> idxEntry : desc.indexes().entrySet()) {
             GridQueryIndexDescriptor idx = idxEntry.getValue();
@@ -1961,10 +1960,11 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             if (idx.type() == FULLTEXT) {
                 assert txtIdx == null;
 
-                txtIdx = new QueryEntityIndex();
+                txtIdx = new QueryIndex();
 
-                txtIdx.setType(QueryEntityIndex.Type.FULLTEXT);
-                txtIdx.setFields(new ArrayList<>(idx.fields()));
+                txtIdx.setIndexType(QueryIndexType.FULLTEXT);
+
+                txtIdx.setFieldNames(idx.fields(), true);
                 txtIdx.setName(idxEntry.getKey());
             }
             else {
@@ -1973,10 +1973,17 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
                 for (String fieldName : idx.fields())
                     grp.add(idx.descending(fieldName) ? fieldName + " desc" : fieldName);
 
-                QueryEntityIndex sortedIdx = new QueryEntityIndex();
+                QueryIndex sortedIdx = new QueryIndex();
+
+                sortedIdx.setIndexType(idx.type() == SORTED ? QueryIndexType.SORTED : QueryIndexType.GEOSPATIAL);
+
+                LinkedHashMap<String, Boolean> fields = new LinkedHashMap<>();
+
+                for (String f : idx.fields())
+                    fields.put(f, !idx.descending(f));
+
+                sortedIdx.setFields(fields);
 
-                sortedIdx.setType(idx.type() == SORTED ? QueryEntityIndex.Type.SORTED : QueryEntityIndex.Type.GEOSPATIAL);
-                sortedIdx.setFields(grp);
                 sortedIdx.setName(idxEntry.getKey());
 
                 idxs.add(sortedIdx);
@@ -1985,14 +1992,14 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
         if (desc.valueTextIndex()) {
             if (txtIdx == null) {
-                txtIdx = new QueryEntityIndex();
+                txtIdx = new QueryIndex();
 
-                txtIdx.setType(QueryEntityIndex.Type.FULLTEXT);
+                txtIdx.setIndexType(QueryIndexType.FULLTEXT);
 
-                txtIdx.setFields(Arrays.asList(_VAL));
+                txtIdx.setFieldNames(Arrays.asList(_VAL), true);
             }
             else
-                txtIdx.getFields().add(_VAL);
+                txtIdx.getFields().put(_VAL, true);
         }
 
         if (txtIdx != null)


[11/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
deleted file mode 100644
index 00fc866..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
+++ /dev/null
@@ -1,544 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable.builder;
-
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.Map;
-import java.util.Set;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridArgumentCheck;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-import org.apache.ignite.internal.portable.*;
-import org.apache.ignite.internal.processors.cache.portable.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.portable.*;
-
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.HASH_CODE_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROTO_VER;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.PROTO_VER_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.TYPE_ID_POS;
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
-
-/**
- *
- */
-public class PortableBuilderImpl implements PortableBuilder {
-    /** */
-    private static final Object REMOVED_FIELD_MARKER = new Object();
-
-    /** */
-    private final PortableContext ctx;
-
-    /** */
-    private final int typeId;
-
-    /** May be null. */
-    private String typeName;
-
-    /** May be null. */
-    private String clsNameToWrite;
-
-    /** */
-    private boolean registeredType = true;
-
-    /** */
-    private Map<String, Object> assignedVals;
-
-    /** */
-    private Map<Integer, Object> readCache;
-
-    /** Position of object in source array, or -1 if object is not created from PortableObject. */
-    private final int start;
-
-    /** Total header length */
-    private final int hdrLen;
-
-    /**
-     * Context of PortableObject reading process. Or {@code null} if object is not created from PortableObject.
-     */
-    private final PortableBuilderReader reader;
-
-    /** */
-    private int hashCode;
-
-    /**
-     * @param clsName Class name.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, String clsName) {
-        this(ctx, ctx.typeId(clsName), PortableContext.typeName(clsName));
-    }
-
-    /**
-     * @param typeId Type ID.
-     * @param ctx Portable context.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId) {
-        this(ctx, typeId, null);
-    }
-
-    /**
-     * @param typeName Type name.
-     * @param ctx Context.
-     * @param typeId Type id.
-     */
-    public PortableBuilderImpl(PortableContext ctx, int typeId, String typeName) {
-        this.typeId = typeId;
-        this.typeName = typeName;
-        this.ctx = ctx;
-
-        start = -1;
-        reader = null;
-        hdrLen = DFLT_HDR_LEN;
-
-        readCache = Collections.emptyMap();
-    }
-
-    /**
-     * @param obj Object to wrap.
-     */
-    public PortableBuilderImpl(PortableObjectImpl obj) {
-        this(new PortableBuilderReader(obj), obj.start());
-
-        reader.registerObject(this);
-    }
-
-    /**
-     * @param reader ctx
-     * @param start Start.
-     */
-    PortableBuilderImpl(PortableBuilderReader reader, int start) {
-        this.reader = reader;
-        this.start = start;
-
-        byte ver = reader.readByteAbsolute(start + PROTO_VER_POS);
-
-        PortableUtils.checkProtocolVersion(ver);
-
-        int typeId = reader.readIntAbsolute(start + TYPE_ID_POS);
-        ctx = reader.portableContext();
-        hashCode = reader.readIntAbsolute(start + HASH_CODE_POS);
-
-        if (typeId == UNREGISTERED_TYPE_ID) {
-            int mark = reader.position();
-
-            reader.position(start + CLS_NAME_POS);
-
-            clsNameToWrite = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: IGNITE-1272 - Is class loader needed here?
-                cls = U.forName(clsNameToWrite, null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsNameToWrite, e);
-            }
-
-            this.typeId = ctx.descriptorForClass(cls).typeId();
-
-            registeredType = false;
-
-            hdrLen = reader.position() - mark;
-
-            reader.position(mark);
-        }
-        else {
-            this.typeId = typeId;
-            hdrLen = DFLT_HDR_LEN;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject build() {
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(ctx, 0, typeId, false)) {
-
-            PortableBuilderSerializer serializationCtx = new PortableBuilderSerializer();
-
-            serializationCtx.registerObjectWriting(this, 0);
-
-            serializeTo(writer, serializationCtx);
-
-            byte[] arr = writer.array();
-
-            return new PortableObjectImpl(ctx, arr, 0);
-        }
-    }
-
-    /**
-     * @param writer Writer.
-     * @param serializer Serializer.
-     */
-    void serializeTo(PortableWriterExImpl writer, PortableBuilderSerializer serializer) {
-        writer.doWriteByte(GridPortableMarshaller.OBJ);
-        writer.doWriteByte(PROTO_VER);
-        writer.doWriteBoolean(true);
-        writer.doWriteInt(registeredType ? typeId : UNREGISTERED_TYPE_ID);
-        writer.doWriteInt(hashCode);
-
-        // Length and raw offset.
-        writer.reserve(8);
-
-        if (!registeredType)
-            writer.writeString(clsNameToWrite);
-
-        Set<Integer> remainsFlds = null;
-
-        if (reader != null) {
-            Map<Integer, Object> assignedFldsById;
-
-            if (assignedVals != null) {
-                assignedFldsById = U.newHashMap(assignedVals.size());
-
-                for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                    int fldId = ctx.fieldId(typeId, entry.getKey());
-
-                    assignedFldsById.put(fldId, entry.getValue());
-                }
-
-                remainsFlds = assignedFldsById.keySet();
-            }
-            else
-                assignedFldsById = Collections.emptyMap();
-
-            int rawOff = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            reader.position(start + hdrLen);
-
-            int cpStart = -1;
-
-            while (reader.position() < rawOff) {
-                int fldId = reader.readInt();
-
-                int len = reader.readInt();
-
-                if (assignedFldsById.containsKey(fldId)) {
-                    if (cpStart >= 0) {
-                        writer.write(reader.array(), cpStart, reader.position() - 4 - 4 - cpStart);
-
-                        cpStart = -1;
-                    }
-
-                    Object assignedVal = assignedFldsById.remove(fldId);
-
-                    reader.skip(len);
-
-                    if (assignedVal != REMOVED_FIELD_MARKER) {
-                        writer.writeInt(fldId);
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, assignedVal);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-                else {
-                    int type = len != 0 ? reader.readByte(0) : 0;
-
-                    if (len != 0 && !PortableUtils.isPlainArrayType(type) && PortableUtils.isPlainType(type)) {
-                        if (cpStart < 0)
-                            cpStart = reader.position() - 4 - 4;
-
-                        reader.skip(len);
-                    }
-                    else {
-                        if (cpStart >= 0) {
-                            writer.write(reader.array(), cpStart, reader.position() - 4 - cpStart);
-
-                            cpStart = -1;
-                        }
-                        else
-                            writer.writeInt(fldId);
-
-                        Object val;
-
-                        if (len == 0)
-                            val = null;
-                        else if (readCache == null) {
-                            int savedPos = reader.position();
-
-                            val = reader.parseValue();
-
-                            assert reader.position() == savedPos + len;
-                        }
-                        else {
-                            val = readCache.get(fldId);
-
-                            reader.skip(len);
-                        }
-
-                        int lenPos = writer.reserveAndMark(4);
-
-                        serializer.writeValue(writer, val);
-
-                        writer.writeDelta(lenPos);
-                    }
-                }
-            }
-
-            if (cpStart >= 0)
-                writer.write(reader.array(), cpStart, reader.position() - cpStart);
-        }
-
-        if (assignedVals != null && (remainsFlds == null || !remainsFlds.isEmpty())) {
-            boolean metadataEnabled = ctx.isMetaDataEnabled(typeId);
-
-            PortableMetadata metadata = null;
-
-            if (metadataEnabled)
-                metadata = ctx.metaData(typeId);
-
-            Map<String, String> newFldsMetadata = null;
-
-            for (Map.Entry<String, Object> entry : assignedVals.entrySet()) {
-                Object val = entry.getValue();
-
-                if (val == REMOVED_FIELD_MARKER)
-                    continue;
-
-                String name = entry.getKey();
-
-                int fldId = ctx.fieldId(typeId, name);
-
-                if (remainsFlds != null && !remainsFlds.contains(fldId))
-                    continue;
-
-                writer.writeInt(fldId);
-
-                int lenPos = writer.reserveAndMark(4);
-
-                serializer.writeValue(writer, val);
-
-                writer.writeDelta(lenPos);
-
-                if (metadataEnabled) {
-                    String oldFldTypeName = metadata == null ? null : metadata.fieldTypeName(name);
-
-                    String newFldTypeName;
-
-                    if (val instanceof PortableValueWithType)
-                        newFldTypeName = ((PortableValueWithType)val).typeName();
-                    else {
-                        byte type = PortableUtils.typeByClass(val.getClass());
-
-                        newFldTypeName = CacheObjectPortableProcessorImpl.fieldTypeName(type);
-                    }
-
-                    if (oldFldTypeName == null) {
-                        // It's a new field, we have to add it to metadata.
-
-                        if (newFldsMetadata == null)
-                            newFldsMetadata = new HashMap<>();
-
-                        newFldsMetadata.put(name, newFldTypeName);
-                    }
-                    else {
-                        if (!"Object".equals(oldFldTypeName) && !oldFldTypeName.equals(newFldTypeName)) {
-                            throw new PortableException(
-                                "Wrong value has been set [" +
-                                    "typeName=" + (typeName == null ? metadata.typeName() : typeName) +
-                                    ", fieldName=" + name +
-                                    ", fieldType=" + oldFldTypeName +
-                                    ", assignedValueType=" + newFldTypeName +
-                                    ", assignedValue=" + (((PortableValueWithType)val).value()) + ']'
-                            );
-                        }
-                    }
-                }
-            }
-
-            if (newFldsMetadata != null) {
-                String typeName = this.typeName;
-
-                if (typeName == null)
-                    typeName = metadata.typeName();
-
-                ctx.updateMetaData(typeId, typeName, newFldsMetadata);
-            }
-        }
-
-        writer.writeRawOffsetIfNeeded();
-
-        if (reader != null) {
-            int rawOff = reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-            int len = reader.readIntAbsolute(start + TOTAL_LEN_POS);
-
-            if (rawOff < len)
-                writer.write(reader.array(), rawOff, len - rawOff);
-        }
-
-        writer.writeLength();
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilderImpl hashCode(int hashCode) {
-        this.hashCode = hashCode;
-
-        return this;
-    }
-
-    /**
-     *
-     */
-    private void ensureReadCacheInit() {
-        if (readCache == null) {
-            Map<Integer, Object> readCache = new HashMap<>();
-
-            int pos = start + hdrLen;
-            int end = start + reader.readIntAbsolute(start + RAW_DATA_OFF_POS);
-
-            while (pos < end) {
-                int fieldId = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                int len = reader.readIntAbsolute(pos);
-
-                pos += 4;
-
-                Object val = reader.getValueQuickly(pos, len);
-
-                readCache.put(fieldId, val);
-
-                pos += len;
-            }
-
-            this.readCache = readCache;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <F> F getField(String name) {
-        Object val;
-
-        if (assignedVals != null && assignedVals.containsKey(name)) {
-            val = assignedVals.get(name);
-
-            if (val == REMOVED_FIELD_MARKER)
-                return null;
-        }
-        else {
-            ensureReadCacheInit();
-
-            int fldId = ctx.fieldId(typeId, name);
-
-            val = readCache.get(fldId);
-        }
-
-        return (F)PortableUtils.unwrapLazy(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, Object val) {
-        GridArgumentCheck.notNull(val, name);
-
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        Object oldVal = assignedVals.put(name, val);
-
-        if (oldVal instanceof PortableValueWithType) {
-            ((PortableValueWithType)oldVal).value(val);
-
-            assignedVals.put(name, oldVal);
-        }
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        //int fldId = ctx.fieldId(typeId, fldName);
-
-        assignedVals.put(name, new PortableValueWithType(PortableUtils.typeByClass(type), val));
-
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder setField(String name, @Nullable PortableBuilder builder) {
-        if (builder == null)
-            return setField(name, null, Object.class);
-        else
-            return setField(name, (Object)builder);
-    }
-
-    /**
-     * Removes field from portable object.
-     *
-     * @param name Field name.
-     * @return {@code this} instance for chaining.
-     */
-    @Override public PortableBuilderImpl removeField(String name) {
-        if (assignedVals == null)
-            assignedVals = new LinkedHashMap<>();
-
-        assignedVals.put(name, REMOVED_FIELD_MARKER);
-
-        return this;
-    }
-
-    /**
-     * Creates builder initialized by specified portable object.
-     *
-     * @param obj Portable object to initialize builder.
-     * @return New builder.
-     */
-    public static PortableBuilderImpl wrap(PortableObject obj) {
-        PortableObjectImpl heapObj;
-
-        if (obj instanceof PortableObjectOffheapImpl)
-            heapObj = (PortableObjectImpl)((PortableObjectOffheapImpl)obj).heapCopy();
-        else
-            heapObj = (PortableObjectImpl)obj;
-
-        return new PortableBuilderImpl(heapObj);
-    }
-
-    /**
-     * @return Object start position in source array.
-     */
-    int start() {
-        return start;
-    }
-
-    /**
-     * @return Object type id.
-     */
-    public int typeId() {
-        return typeId;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index afa40a3..6404a69 100644
--- 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
@@ -23,12 +23,12 @@ 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.IgniteObjectImpl;
 import org.apache.ignite.internal.portable.PortablePrimitives;
-import org.apache.ignite.internal.portable.PortableReaderExImpl;
+import org.apache.ignite.internal.portable.IgniteObjectReaderExImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.portable.PortableWriterExImpl;
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.IgniteObjectWriterExImpl;
+import org.apache.ignite.igniteobject.IgniteObjectException;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
@@ -42,13 +42,13 @@ class PortableBuilderReader {
     private static final PortablePrimitives PRIM = PortablePrimitives.get();
 
     /** */
-    private final Map<Integer, PortableBuilderImpl> objMap = new HashMap<>();
+    private final Map<Integer, IgniteObjectBuilderImpl> objMap = new HashMap<>();
 
     /** */
     private final PortableContext ctx;
 
     /** */
-    private final PortableReaderExImpl reader;
+    private final IgniteObjectReaderExImpl reader;
 
     /** */
     private byte[] arr;
@@ -59,13 +59,13 @@ class PortableBuilderReader {
     /**
      * @param objImpl Portable object
      */
-    PortableBuilderReader(PortableObjectImpl objImpl) {
+    PortableBuilderReader(IgniteObjectImpl 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);
+        reader = new IgniteObjectReaderExImpl(portableContext(), arr, pos, null);
     }
 
     /**
@@ -78,7 +78,7 @@ class PortableBuilderReader {
     /**
      * @param obj Mutable portable object.
      */
-    public void registerObject(PortableBuilderImpl obj) {
+    public void registerObject(IgniteObjectBuilderImpl obj) {
         objMap.put(obj.start(), obj);
     }
 
@@ -170,7 +170,7 @@ class PortableBuilderReader {
             return null;
 
         if (flag != STRING)
-            throw new PortableException("Failed to deserialize String.");
+            throw new IgniteObjectException("Failed to deserialize String.");
 
         boolean convert = readBoolean();
         int len = readInt();
@@ -338,7 +338,7 @@ class PortableBuilderReader {
                 break;
 
             default:
-                throw new PortableException("Invalid flag value: " + type);
+                throw new IgniteObjectException("Invalid flag value: " + type);
         }
 
         pos += len;
@@ -359,10 +359,10 @@ class PortableBuilderReader {
             case GridPortableMarshaller.HANDLE: {
                 int objStart = pos - readIntAbsolute(pos + 1);
 
-                PortableBuilderImpl res = objMap.get(objStart);
+                IgniteObjectBuilderImpl res = objMap.get(objStart);
 
                 if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
+                    res = new IgniteObjectBuilderImpl(this, objStart);
 
                     objMap.put(objStart, res);
                 }
@@ -371,10 +371,10 @@ class PortableBuilderReader {
             }
 
             case GridPortableMarshaller.OBJ: {
-                PortableBuilderImpl res = objMap.get(pos);
+                IgniteObjectBuilderImpl res = objMap.get(pos);
 
                 if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
+                    res = new IgniteObjectBuilderImpl(this, pos);
 
                     objMap.put(pos, res);
                 }
@@ -455,13 +455,13 @@ class PortableBuilderReader {
 
                 int start = readIntAbsolute(pos + 4 + size);
 
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr, pos + 4 + start);
+                IgniteObjectImpl portableObj = new IgniteObjectImpl(ctx, arr, pos + 4 + start);
 
                 return new PortablePlainPortableObject(portableObj);
             }
 
             default:
-                throw new PortableException("Invalid flag value: " + type);
+                throw new IgniteObjectException("Invalid flag value: " + type);
         }
     }
 
@@ -484,10 +484,10 @@ class PortableBuilderReader {
             case GridPortableMarshaller.HANDLE: {
                 int objStart = pos - 1 - readInt();
 
-                PortableBuilderImpl res = objMap.get(objStart);
+                IgniteObjectBuilderImpl res = objMap.get(objStart);
 
                 if (res == null) {
-                    res = new PortableBuilderImpl(this, objStart);
+                    res = new IgniteObjectBuilderImpl(this, objStart);
 
                     objMap.put(objStart, res);
                 }
@@ -498,10 +498,10 @@ class PortableBuilderReader {
             case GridPortableMarshaller.OBJ: {
                 pos--;
 
-                PortableBuilderImpl res = objMap.get(pos);
+                IgniteObjectBuilderImpl res = objMap.get(pos);
 
                 if (res == null) {
-                    res = new PortableBuilderImpl(this, pos);
+                    res = new IgniteObjectBuilderImpl(this, pos);
 
                     objMap.put(pos, res);
                 }
@@ -633,7 +633,7 @@ class PortableBuilderReader {
                     if (flag == GridPortableMarshaller.NULL) continue;
 
                     if (flag != GridPortableMarshaller.DATE)
-                        throw new PortableException("Invalid flag value: " + flag);
+                        throw new IgniteObjectException("Invalid flag value: " + flag);
 
                     long time = PRIM.readLong(arr, pos);
 
@@ -657,7 +657,7 @@ class PortableBuilderReader {
                         continue;
 
                     if (flag != GridPortableMarshaller.TIMESTAMP)
-                        throw new PortableException("Invalid flag value: " + flag);
+                        throw new IgniteObjectException("Invalid flag value: " + flag);
 
                     long time = PRIM.readLong(arr, pos);
 
@@ -719,7 +719,7 @@ class PortableBuilderReader {
                         return new PortableLazySet(this, size);
                 }
 
-                throw new PortableException("Unknown collection type: " + colType);
+                throw new IgniteObjectException("Unknown collection type: " + colType);
             }
 
             case GridPortableMarshaller.MAP:
@@ -741,7 +741,7 @@ class PortableBuilderReader {
 
                 int start = readInt();
 
-                PortableObjectImpl portableObj = new PortableObjectImpl(ctx, arr,
+                IgniteObjectImpl portableObj = new IgniteObjectImpl(ctx, arr,
                     pos - 4 - size + start);
 
                 return new PortablePlainPortableObject(portableObj);
@@ -749,7 +749,7 @@ class PortableBuilderReader {
 
 
             default:
-                throw new PortableException("Invalid flag value: " + type);
+                throw new IgniteObjectException("Invalid flag value: " + type);
         }
 
         PortableAbstractLazyValue res;
@@ -795,7 +795,7 @@ class PortableBuilderReader {
     /**
      * @return Reader.
      */
-    PortableReaderExImpl reader() {
+    IgniteObjectReaderExImpl reader() {
         return reader;
     }
 
@@ -830,7 +830,7 @@ class PortableBuilderReader {
         }
 
         /** {@inheritDoc} */
-        @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+        @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
             ctx.writeValue(writer, wrappedCollection());
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 976059a..54e9150 100644
--- 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
@@ -27,5 +27,5 @@ interface PortableBuilderSerializationAware {
      * @param writer Writer.
      * @param ctx Context.
      */
-    public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx);
+    public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 2d9c961..8ea87c9 100644
--- 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
@@ -17,10 +17,11 @@
 
 package org.apache.ignite.internal.portable.builder;
 
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableObjectEx;
+import org.apache.ignite.internal.portable.IgniteObjectEx;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.portable.*;
 
@@ -31,16 +32,16 @@ import java.util.*;
  */
 class PortableBuilderSerializer {
     /** */
-    private final Map<PortableBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
+    private final Map<IgniteObjectBuilderImpl, Integer> objToPos = new IdentityHashMap<>();
 
     /** */
-    private Map<PortableObject, PortableBuilderImpl> portableObjToWrapper;
+    private Map<IgniteObject, IgniteObjectBuilderImpl> portableObjToWrapper;
 
     /**
      * @param obj Mutable object.
      * @param posInResArr Object position in the array.
      */
-    public void registerObjectWriting(PortableBuilderImpl obj, int posInResArr) {
+    public void registerObjectWriting(IgniteObjectBuilderImpl obj, int posInResArr) {
         objToPos.put(obj, posInResArr);
     }
 
@@ -48,7 +49,7 @@ class PortableBuilderSerializer {
      * @param writer Writer.
      * @param val Value.
      */
-    public void writeValue(PortableWriterExImpl writer, Object val) {
+    public void writeValue(IgniteObjectWriterExImpl writer, Object val) {
         if (val == null) {
             writer.writeByte(GridPortableMarshaller.NULL);
 
@@ -61,23 +62,23 @@ class PortableBuilderSerializer {
             return;
         }
 
-        if (val instanceof PortableObjectEx) {
+        if (val instanceof IgniteObjectEx) {
             if (portableObjToWrapper == null)
                 portableObjToWrapper = new IdentityHashMap<>();
 
-            PortableBuilderImpl wrapper = portableObjToWrapper.get(val);
+            IgniteObjectBuilderImpl wrapper = portableObjToWrapper.get(val);
 
             if (wrapper == null) {
-                wrapper = PortableBuilderImpl.wrap((PortableObject)val);
+                wrapper = IgniteObjectBuilderImpl.wrap((IgniteObject)val);
 
-                portableObjToWrapper.put((PortableObject)val, wrapper);
+                portableObjToWrapper.put((IgniteObject)val, wrapper);
             }
 
             val = wrapper;
         }
 
-        if (val instanceof PortableBuilderImpl) {
-            PortableBuilderImpl obj = (PortableBuilderImpl)val;
+        if (val instanceof IgniteObjectBuilderImpl) {
+            IgniteObjectBuilderImpl obj = (IgniteObjectBuilderImpl)val;
 
             Integer posInResArr = objToPos.get(obj);
 
@@ -186,7 +187,7 @@ class PortableBuilderSerializer {
      * @param arr The array.
      * @param compTypeId Component type ID.
      */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
+    public void writeArray(IgniteObjectWriterExImpl writer, byte elementType, Object[] arr, int compTypeId) {
         writer.writeByte(elementType);
         writer.writeInt(compTypeId);
         writer.writeInt(arr.length);
@@ -201,7 +202,7 @@ class PortableBuilderSerializer {
      * @param arr The array.
      * @param clsName Component class name.
      */
-    public void writeArray(PortableWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
+    public void writeArray(IgniteObjectWriterExImpl writer, byte elementType, Object[] arr, String clsName) {
         writer.writeByte(elementType);
         writer.writeInt(GridPortableMarshaller.UNREGISTERED_TYPE_ID);
         writer.writeString(clsName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index d864a6e..62c4eb1 100644
--- 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
@@ -18,10 +18,10 @@
 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.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
 
 /**
  *
@@ -54,7 +54,7 @@ class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
                 cls = U.forName(reader.readString(), null);
             }
             catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+                throw new IgniteObjectInvalidClassException("Failed to load the class: " + clsName, e);
             }
 
             compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
@@ -90,7 +90,7 @@ class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
                 continue;
 
             if (flag != GridPortableMarshaller.ENUM)
-                throw new PortableException("Invalid flag value: " + flag);
+                throw new IgniteObjectException("Invalid flag value: " + flag);
 
             res[i] = new PortableBuilderEnum(reader);
         }
@@ -99,7 +99,7 @@ class PortableEnumArrayLazyValue extends PortableAbstractLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (val != null) {
             if (clsName != null)
                 ctx.writeArray(writer, GridPortableMarshaller.ENUM_ARR, (Object[])val, clsName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index a08cfdd..098a70a 100644
--- 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
@@ -131,7 +131,7 @@ class PortableLazyArrayList extends AbstractList<Object> implements PortableBuil
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (delegate == null) {
             int size = reader.readIntAbsolute(off + 1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index f793d7a..998be43 100644
--- 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
@@ -188,7 +188,7 @@ class PortableLazyLinkedList extends AbstractList<Object> implements PortableBui
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (delegate == null) {
             int size = reader.readIntAbsolute(off + 1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 12cbfd6..c1c2c5a 100644
--- 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
@@ -85,7 +85,7 @@ class PortableLazyMap extends AbstractMap<Object, Object> implements PortableBui
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (delegate == null) {
             int size = reader.readIntAbsolute(off + 1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index bd027f5..8ce4cc1 100644
--- 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
@@ -59,7 +59,7 @@ class PortableLazyMapEntry implements Map.Entry<Object, Object>, PortableBuilder
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         writer.writeByte(GridPortableMarshaller.MAP_ENTRY);
 
         ctx.writeValue(writer, key);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 16772af..2f9a34b 100644
--- 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
@@ -21,7 +21,7 @@ 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.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
@@ -47,7 +47,7 @@ class PortableLazySet extends PortableAbstractLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (val == null) {
             int size = reader.readIntAbsolute(off + 1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 09fb844..92bd4b4 100644
--- 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
@@ -43,7 +43,7 @@ public class PortableModifiableLazyValue extends PortableAbstractLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (val == null)
             writer.write(reader.array(), valOff, len);
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 1126a3c..b7f2820 100644
--- 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
@@ -18,9 +18,9 @@
 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.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
 
 /**
  *
@@ -53,7 +53,7 @@ class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
                 cls = U.forName(reader.readString(), null);
             }
             catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
+                throw new IgniteObjectInvalidClassException("Failed to load the class: " + clsName, e);
             }
 
             compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
@@ -82,7 +82,7 @@ class PortableObjectArrayLazyValue extends PortableAbstractLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (clsName == null)
             ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 136958a..a51820a 100644
--- 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
@@ -43,7 +43,7 @@ class PortablePlainLazyValue extends PortableAbstractLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         writer.write(reader.array(), valOff, len);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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
index 8743fbe..5bbe3ca 100644
--- 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
@@ -17,22 +17,22 @@
 
 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;
+import org.apache.ignite.internal.portable.IgniteObjectImpl;
+import org.apache.ignite.internal.portable.IgniteObjectOffheapImpl;
+import org.apache.ignite.internal.portable.IgniteObjectWriterExImpl;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
  */
 public class PortablePlainPortableObject implements PortableLazyValue {
     /** */
-    private final PortableObject portableObj;
+    private final IgniteObject portableObj;
 
     /**
      * @param portableObj Portable object.
      */
-    public PortablePlainPortableObject(PortableObject portableObj) {
+    public PortablePlainPortableObject(IgniteObject portableObj) {
         this.portableObj = portableObj;
     }
 
@@ -42,12 +42,12 @@ public class PortablePlainPortableObject implements PortableLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
-        PortableObject val = portableObj;
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
+        IgniteObject val = portableObj;
 
-        if (val instanceof PortableObjectOffheapImpl)
-            val = ((PortableObjectOffheapImpl)val).heapCopy();
+        if (val instanceof IgniteObjectOffheapImpl)
+            val = ((IgniteObjectOffheapImpl)val).heapCopy();
 
-        writer.doWritePortableObject((PortableObjectImpl)val);
+        writer.doWritePortableObject((IgniteObjectImpl)val);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
index 2e031f0..21d25c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableValueWithType.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.portable.builder;
 
-import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.portable.IgniteObjectWriterExImpl;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -41,7 +41,7 @@ class PortableValueWithType implements PortableLazyValue {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeTo(PortableWriterExImpl writer, PortableBuilderSerializer ctx) {
+    @Override public void writeTo(IgniteObjectWriterExImpl writer, PortableBuilderSerializer ctx) {
         if (val instanceof PortableBuilderSerializationAware)
             ((PortableBuilderSerializationAware)val).writeTo(writer, ctx);
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
index 107b02e..af86bfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.portable.streams;
 
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.igniteobject.IgniteObjectException;
 
 /**
  * Portable abstract input stream.
@@ -268,7 +268,7 @@ public abstract class PortableAbstractInputStream extends PortableAbstractStream
     /** {@inheritDoc} */
     @Override public void position(int pos) {
         if (remaining() + this.pos < pos)
-            throw new PortableException("Position is out of bounds: " + pos);
+            throw new IgniteObjectException("Position is out of bounds: " + pos);
         else
             this.pos = pos;
     }
@@ -285,7 +285,7 @@ public abstract class PortableAbstractInputStream extends PortableAbstractStream
      */
     protected void ensureEnoughData(int cnt) {
         if (remaining() < cnt)
-            throw new PortableException("Not enough data to read the value [position=" + pos +
+            throw new IgniteObjectException("Not enough data to read the value [position=" + pos +
                 ", requiredBytes=" + cnt + ", remainingBytes=" + remaining() + ']');
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index d889902..3e311dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -53,7 +53,7 @@ public interface CacheObject extends Message {
     /**
      * Gets flag indicating whether object value is a platform type. Platform types will be automatically
      * deserialized on public API cache operations regardless whether
-     * {@link org.apache.ignite.IgniteCache#withKeepPortable()} is used or not.
+     * {@link org.apache.ignite.IgniteCache#withKeepBinary()} is used or not.
      *
      * @return Platform type flag.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f39084e..9838158 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -311,7 +311,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
+    @Override public <K1, V1> IgniteCache<K1, V1> withKeepBinary() {
         return keepPortable();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
index 23edd9e..5ead014 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
@@ -21,7 +21,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -43,8 +43,8 @@ public class CacheDefaultPortableAffinityKeyMapper extends GridCacheDefaultAffin
             U.error(log, "Failed to marshal key to portable: " + key, e);
         }
 
-        if (key instanceof PortableObject)
-            return proc.affinityKey((PortableObject)key);
+        if (key instanceof IgniteObject)
+            return proc.affinityKey((IgniteObject)key);
         else
             return super.affinityKey(key);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
index b22774b..e1dde75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
@@ -17,17 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.portable;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
index fcd73d2..b7e942c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.cache.portable;
 import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -35,13 +35,13 @@ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor
      * @param typeId Type ID.
      * @return Builder.
      */
-    public PortableBuilder builder(int typeId);
+    public IgniteObjectBuilder builder(int typeId);
 
     /**
      * @param clsName Class name.
      * @return Builder.
      */
-    public PortableBuilder builder(String clsName);
+    public IgniteObjectBuilder builder(String clsName);
 
     /**
      * Creates builder initialized by existing portable object.
@@ -49,14 +49,14 @@ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor
      * @param portableObj Portable object to edit.
      * @return Portable builder.
      */
-    public PortableBuilder builder(PortableObject portableObj);
+    public IgniteObjectBuilder builder(IgniteObject portableObj);
 
     /**
      * @param typeId Type ID.
      * @param newMeta New meta data.
      * @throws IgniteException In case of error.
      */
-    public void addMeta(int typeId, final PortableMetadata newMeta) throws IgniteException;
+    public void addMeta(int typeId, final IgniteObjectMetadata newMeta) throws IgniteException;
 
     /**
      * @param typeId Type ID.
@@ -73,26 +73,26 @@ public interface CacheObjectPortableProcessor extends IgniteCacheObjectProcessor
      * @return Meta data.
      * @throws IgniteException In case of error.
      */
-    @Nullable public PortableMetadata metadata(int typeId) throws IgniteException;
+    @Nullable public IgniteObjectMetadata metadata(int typeId) throws IgniteException;
 
     /**
      * @param typeIds Type ID.
      * @return Meta data.
      * @throws IgniteException In case of error.
      */
-    public Map<Integer, PortableMetadata> metadata(Collection<Integer> typeIds) throws IgniteException;
+    public Map<Integer, IgniteObjectMetadata> metadata(Collection<Integer> typeIds) throws IgniteException;
 
     /**
      * @return Metadata for all types.
      * @throws IgniteException In case of error.
      */
-    public Collection<PortableMetadata> metadata() throws IgniteException;
+    public Collection<IgniteObjectMetadata> metadata() throws IgniteException;
 
     /**
      * @return Portables interface.
      * @throws IgniteException If failed.
      */
-    public IgnitePortables portables() throws IgniteException;
+    public IgniteObjects portables() throws IgniteException;
 
     /**
      * @param obj Original object.

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/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 20a7eb5..85134bf 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
@@ -39,7 +39,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -47,11 +47,11 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
-import org.apache.ignite.internal.portable.PortableMetaDataImpl;
-import org.apache.ignite.internal.portable.PortableObjectImpl;
-import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
+import org.apache.ignite.internal.portable.IgniteObjectMetaDataImpl;
+import org.apache.ignite.internal.portable.IgniteObjectImpl;
+import org.apache.ignite.internal.portable.IgniteObjectOffheapImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
-import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
+import org.apache.ignite.internal.portable.builder.IgniteObjectBuilderImpl;
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOffheapInputStream;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -81,10 +81,10 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;
@@ -141,10 +141,10 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     private final boolean clientNode;
 
     /** */
-    private volatile IgniteCacheProxy<PortableMetaDataKey, PortableMetadata> metaDataCache;
+    private volatile IgniteCacheProxy<PortableMetaDataKey, IgniteObjectMetadata> metaDataCache;
 
     /** */
-    private final ConcurrentHashMap8<PortableMetaDataKey, PortableMetadata> clientMetaDataCache;
+    private final ConcurrentHashMap8<PortableMetaDataKey, IgniteObjectMetadata> clientMetaDataCache;
 
     /** Predicate to filter portable meta data in utility cache. */
     private final CacheEntryPredicate metaPred = new CacheEntryPredicateAdapter() {
@@ -166,10 +166,10 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
     /** */
     @GridToStringExclude
-    private IgnitePortables portables;
+    private IgniteObjects portables;
 
     /** Metadata updates collected before metadata cache is initialized. */
-    private final Map<Integer, PortableMetadata> metaBuf = new ConcurrentHashMap<>();
+    private final Map<Integer, IgniteObjectMetadata> metaBuf = new ConcurrentHashMap<>();
 
     /** */
     private UUID metaCacheQryId;
@@ -268,24 +268,24 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
         clientNode = this.ctx.clientNode();
 
-        clientMetaDataCache = clientNode ? new ConcurrentHashMap8<PortableMetaDataKey, PortableMetadata>() : null;
+        clientMetaDataCache = clientNode ? new ConcurrentHashMap8<PortableMetaDataKey, IgniteObjectMetadata>() : null;
     }
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         if (marsh instanceof PortableMarshaller) {
             PortableMetaDataHandler metaHnd = new PortableMetaDataHandler() {
-                @Override public void addMeta(int typeId, PortableMetadata newMeta)
-                    throws PortableException {
+                @Override public void addMeta(int typeId, IgniteObjectMetadata newMeta)
+                    throws IgniteObjectException {
                     if (metaDataCache == null) {
-                        PortableMetadata oldMeta = metaBuf.get(typeId);
+                        IgniteObjectMetadata oldMeta = metaBuf.get(typeId);
 
                         if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) {
                             synchronized (this) {
                                 Map<String, String> fields = new HashMap<>();
 
                                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                                    newMeta = new PortableMetaDataImpl(newMeta.typeName(),
+                                    newMeta = new IgniteObjectMetaDataImpl(newMeta.typeName(),
                                         fields,
                                         newMeta.affinityKeyFieldName());
 
@@ -307,7 +307,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
                     CacheObjectPortableProcessorImpl.this.addMeta(typeId, newMeta);
                 }
 
-                @Override public PortableMetadata metadata(int typeId) throws PortableException {
+                @Override public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException {
                     if (metaDataCache == null)
                         U.awaitQuiet(startLatch);
 
@@ -317,13 +317,13 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
             PortableMarshaller pMarh0 = (PortableMarshaller)marsh;
 
-            portableCtx = new PortableContext(metaHnd, ctx.gridName());
+            portableCtx = new PortableContext(metaHnd, ctx.config());
 
             IgniteUtils.invoke(PortableMarshaller.class, pMarh0, "setPortableContext", portableCtx);
 
             portableMarsh = new GridPortableMarshaller(portableCtx);
 
-            portables = new IgnitePortablesImpl(ctx, this);
+            portables = new IgniteObjectsImpl(ctx, this);
         }
     }
 
@@ -350,7 +350,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
                 GridCacheQueryManager qryMgr = metaDataCache.context().queries();
 
-                CacheQuery<Map.Entry<PortableMetaDataKey, PortableMetadata>> qry =
+                CacheQuery<Map.Entry<PortableMetaDataKey, IgniteObjectMetadata>> qry =
                     qryMgr.createScanQuery(new MetaDataPredicate(), null, false);
 
                 qry.keepAll(false);
@@ -358,9 +358,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
                 qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
 
                 try {
-                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, PortableMetadata>> fut = qry.execute();
+                    CacheQueryFuture<Map.Entry<PortableMetaDataKey, IgniteObjectMetadata>> fut = qry.execute();
 
-                    Map.Entry<PortableMetaDataKey, PortableMetadata> next;
+                    Map.Entry<PortableMetaDataKey, IgniteObjectMetadata> next;
 
                     while ((next = fut.next()) != null) {
                         assert next.getKey() != null : next;
@@ -382,7 +382,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
         startLatch.countDown();
 
-        for (Map.Entry<Integer, PortableMetadata> e : metaBuf.entrySet())
+        for (Map.Entry<Integer, IgniteObjectMetadata> e : metaBuf.entrySet())
             addMeta(e.getKey(), e.getValue());
 
         metaBuf.clear();
@@ -400,16 +400,16 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
      * @param key Metadata key.
      * @param newMeta Metadata.
      */
-    private void addClientCacheMetaData(PortableMetaDataKey key, final PortableMetadata newMeta) {
+    private void addClientCacheMetaData(PortableMetaDataKey key, final IgniteObjectMetadata newMeta) {
         clientMetaDataCache.compute(key,
-            new ConcurrentHashMap8.BiFun<PortableMetaDataKey, PortableMetadata, PortableMetadata>() {
-                @Override public PortableMetadata apply(PortableMetaDataKey key, PortableMetadata oldMeta) {
-                    PortableMetadata res;
+            new ConcurrentHashMap8.BiFun<PortableMetaDataKey, IgniteObjectMetadata, IgniteObjectMetadata>() {
+                @Override public IgniteObjectMetadata apply(PortableMetaDataKey key, IgniteObjectMetadata oldMeta) {
+                    IgniteObjectMetadata res;
 
                     try {
                         res = checkMeta(key.typeId(), oldMeta, newMeta, null) ? newMeta : oldMeta;
                     }
-                    catch (PortableException e) {
+                    catch (IgniteObjectException e) {
                         res = oldMeta;
                     }
 
@@ -430,9 +430,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     /**
      * @param obj Object.
      * @return Bytes.
-     * @throws PortableException If failed.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
      */
-    public byte[] marshal(@Nullable Object obj) throws PortableException {
+    public byte[] marshal(@Nullable Object obj) throws IgniteObjectException {
         byte[] arr = portableMarsh.marshal(obj, 0);
 
         assert arr.length > 0;
@@ -444,9 +444,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
      * @param ptr Off-heap pointer.
      * @param forceHeap If {@code true} creates heap-based object.
      * @return Object.
-     * @throws PortableException If failed.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
      */
-    public Object unmarshal(long ptr, boolean forceHeap) throws PortableException {
+    public Object unmarshal(long ptr, boolean forceHeap) throws IgniteObjectException {
         assert ptr > 0 : ptr;
 
         int size = UNSAFE.getInt(ptr);
@@ -467,7 +467,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     }
 
     /** {@inheritDoc} */
-    @Override public Object marshalToPortable(@Nullable Object obj) throws PortableException {
+    @Override public Object marshalToPortable(@Nullable Object obj) throws IgniteObjectException {
         if (obj == null)
             return null;
 
@@ -524,9 +524,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
         Object obj0 = portableMarsh.unmarshal(arr, null);
 
-        assert obj0 instanceof PortableObject;
+        assert obj0 instanceof IgniteObject;
 
-        ((PortableObjectImpl)obj0).detachAllowed(true);
+        ((IgniteObjectImpl)obj0).detachAllowed(true);
 
         return obj0;
     }
@@ -539,50 +539,50 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     }
 
     /** {@inheritDoc} */
-    @Override public PortableBuilder builder(int typeId) {
-        return new PortableBuilderImpl(portableCtx, typeId);
+    @Override public IgniteObjectBuilder builder(int typeId) {
+        return new IgniteObjectBuilderImpl(portableCtx, typeId);
     }
 
     /** {@inheritDoc} */
-    @Override public PortableBuilder builder(String clsName) {
-        return new PortableBuilderImpl(portableCtx, clsName);
+    @Override public IgniteObjectBuilder builder(String clsName) {
+        return new IgniteObjectBuilderImpl(portableCtx, clsName);
     }
 
     /** {@inheritDoc} */
-    @Override public PortableBuilder builder(PortableObject portableObj) {
-        return PortableBuilderImpl.wrap(portableObj);
+    @Override public IgniteObjectBuilder builder(IgniteObject portableObj) {
+        return IgniteObjectBuilderImpl.wrap(portableObj);
     }
 
     /** {@inheritDoc} */
     @Override public void updateMetaData(int typeId, String typeName, @Nullable String affKeyFieldName,
-        Map<String, Integer> fieldTypeIds) throws PortableException {
+        Map<String, Integer> fieldTypeIds) throws IgniteObjectException {
         portableCtx.updateMetaData(typeId,
-            new PortableMetaDataImpl(typeName, fieldTypeNames(fieldTypeIds), affKeyFieldName));
+            new IgniteObjectMetaDataImpl(typeName, fieldTypeNames(fieldTypeIds), affKeyFieldName));
     }
 
     /** {@inheritDoc} */
-    @Override public void addMeta(final int typeId, final PortableMetadata newMeta) throws PortableException {
+    @Override public void addMeta(final int typeId, final IgniteObjectMetadata newMeta) throws IgniteObjectException {
         assert newMeta != null;
 
         final PortableMetaDataKey key = new PortableMetaDataKey(typeId);
 
         try {
-            PortableMetadata oldMeta = metaDataCache.localPeek(key);
+            IgniteObjectMetadata oldMeta = metaDataCache.localPeek(key);
 
             if (oldMeta == null || checkMeta(typeId, oldMeta, newMeta, null)) {
-                PortableException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta));
+                IgniteObjectException err = metaDataCache.invoke(key, new MetaDataProcessor(typeId, newMeta));
 
                 if (err != null)
                     throw err;
             }
         }
         catch (CacheException e) {
-            throw new PortableException("Failed to update meta data for type: " + newMeta.typeName(), e);
+            throw new IgniteObjectException("Failed to update meta data for type: " + newMeta.typeName(), e);
         }
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metadata(final int typeId) throws PortableException {
+    @Nullable @Override public IgniteObjectMetadata metadata(final int typeId) throws IgniteObjectException {
         try {
             if (clientNode)
                 return clientMetaDataCache.get(new PortableMetaDataKey(typeId));
@@ -590,58 +590,58 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
             return metaDataCache.localPeek(new PortableMetaDataKey(typeId));
         }
         catch (CacheException e) {
-            throw new PortableException(e);
+            throw new IgniteObjectException(e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public Map<Integer, PortableMetadata> metadata(Collection<Integer> typeIds)
-        throws PortableException {
+    @Override public Map<Integer, IgniteObjectMetadata> metadata(Collection<Integer> typeIds)
+        throws IgniteObjectException {
         try {
             Collection<PortableMetaDataKey> keys = new ArrayList<>(typeIds.size());
 
             for (Integer typeId : typeIds)
                 keys.add(new PortableMetaDataKey(typeId));
 
-            Map<PortableMetaDataKey, PortableMetadata> meta = metaDataCache.getAll(keys);
+            Map<PortableMetaDataKey, IgniteObjectMetadata> meta = metaDataCache.getAll(keys);
 
-            Map<Integer, PortableMetadata> res = U.newHashMap(meta.size());
+            Map<Integer, IgniteObjectMetadata> res = U.newHashMap(meta.size());
 
-            for (Map.Entry<PortableMetaDataKey, PortableMetadata> e : meta.entrySet())
+            for (Map.Entry<PortableMetaDataKey, IgniteObjectMetadata> e : meta.entrySet())
                 res.put(e.getKey().typeId(), e.getValue());
 
             return res;
         }
         catch (CacheException e) {
-            throw new PortableException(e);
+            throw new IgniteObjectException(e);
         }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public Collection<PortableMetadata> metadata() throws PortableException {
+    @Override public Collection<IgniteObjectMetadata> metadata() throws IgniteObjectException {
         if (clientNode)
             return new ArrayList<>(clientMetaDataCache.values());
 
         return F.viewReadOnly(metaDataCache.entrySetx(metaPred),
-            new C1<Cache.Entry<PortableMetaDataKey, PortableMetadata>, PortableMetadata>() {
+            new C1<Cache.Entry<PortableMetaDataKey, IgniteObjectMetadata>, IgniteObjectMetadata>() {
                 private static final long serialVersionUID = 0L;
 
-                @Override public PortableMetadata apply(
-                    Cache.Entry<PortableMetaDataKey, PortableMetadata> e) {
+                @Override public IgniteObjectMetadata apply(
+                    Cache.Entry<PortableMetaDataKey, IgniteObjectMetadata> e) {
                     return e.getValue();
                 }
             });
     }
 
     /** {@inheritDoc} */
-    @Override public IgnitePortables portables() throws IgniteException {
+    @Override public IgniteObjects portables() throws IgniteException {
         return portables;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isPortableObject(Object obj) {
-        return obj instanceof PortableObject;
+        return obj instanceof IgniteObject;
     }
 
     /** {@inheritDoc} */
@@ -653,9 +653,9 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
      * @param po Portable object.
      * @return Affinity key.
      */
-    public Object affinityKey(PortableObject po) {
+    public Object affinityKey(IgniteObject po) {
         try {
-            PortableMetadata meta = po.metaData();
+            IgniteObjectMetadata meta = po.metaData();
 
             if (meta != null) {
                 String affKeyFieldName = meta.affinityKeyFieldName();
@@ -664,7 +664,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
                     return po.field(affKeyFieldName);
             }
         }
-        catch (PortableException e) {
+        catch (IgniteObjectException e) {
             U.error(log, "Failed to get affinity field from portable object: " + po, e);
         }
 
@@ -676,7 +676,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         if (obj == null)
             return 0;
 
-        return isPortableObject(obj) ? ((PortableObject)obj).typeId() : typeId(obj.getClass().getSimpleName());
+        return isPortableObject(obj) ? ((IgniteObject)obj).typeId() : typeId(obj.getClass().getSimpleName());
     }
 
     /** {@inheritDoc} */
@@ -684,12 +684,12 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         if (obj == null)
             return null;
 
-        return isPortableObject(obj) ? ((PortableObject)obj).field(fieldName) : super.field(obj, fieldName);
+        return isPortableObject(obj) ? ((IgniteObject)obj).field(fieldName) : super.field(obj, fieldName);
     }
 
     /** {@inheritDoc} */
     @Override public boolean hasField(Object obj, String fieldName) {
-        return obj != null && ((PortableObject)obj).hasField(fieldName);
+        return obj != null && ((IgniteObject)obj).hasField(fieldName);
     }
 
     /**
@@ -750,8 +750,8 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         if (((CacheObjectPortableContext)ctx).portableEnabled()) {
             obj = toPortable(obj);
 
-            if (obj instanceof PortableObject)
-                return (PortableObjectImpl)obj;
+            if (obj instanceof IgniteObject)
+                return (IgniteObjectImpl)obj;
         }
 
         return toCacheKeyObject0(obj, userObj);
@@ -768,16 +768,16 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
         obj = toPortable(obj);
 
-        if (obj instanceof PortableObject)
-            return (PortableObjectImpl)obj;
+        if (obj instanceof IgniteObject)
+            return (IgniteObjectImpl)obj;
 
         return toCacheObject0(obj, userObj);
     }
 
     /** {@inheritDoc} */
     @Override public CacheObject toCacheObject(CacheObjectContext ctx, byte type, byte[] bytes) {
-        if (type == PortableObjectImpl.TYPE_PORTABLE)
-            return new PortableObjectImpl(portableContext(), bytes, 0);
+        if (type == IgniteObjectImpl.TYPE_PORTABLE)
+            return new IgniteObjectImpl(portableContext(), bytes, 0);
 
         return super.toCacheObject(ctx, type, bytes);
     }
@@ -790,19 +790,19 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
         Object val = unmarshal(valPtr, !tmp);
 
-        if (val instanceof PortableObjectOffheapImpl)
-            return (PortableObjectOffheapImpl)val;
+        if (val instanceof IgniteObjectOffheapImpl)
+            return (IgniteObjectOffheapImpl)val;
 
         return new CacheObjectImpl(val, null);
     }
 
     /** {@inheritDoc} */
-    @Override public Object unwrapTemporary(GridCacheContext ctx, Object obj) throws PortableException {
+    @Override public Object unwrapTemporary(GridCacheContext ctx, Object obj) throws IgniteObjectException {
         if (!((CacheObjectPortableContext)ctx.cacheObjectContext()).portableEnabled())
             return obj;
 
-        if (obj instanceof PortableObjectOffheapImpl)
-            return ((PortableObjectOffheapImpl)obj).heapCopy();
+        if (obj instanceof IgniteObjectOffheapImpl)
+            return ((IgniteObjectOffheapImpl)obj).heapCopy();
 
         return obj;
     }
@@ -828,20 +828,20 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
      * @param newMeta New meta.
      * @param fields Fields map.
      * @return Whether meta is changed.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    private static boolean checkMeta(int typeId, @Nullable PortableMetadata oldMeta,
-        PortableMetadata newMeta, @Nullable Map<String, String> fields) throws PortableException {
+    private static boolean checkMeta(int typeId, @Nullable IgniteObjectMetadata oldMeta,
+        IgniteObjectMetadata newMeta, @Nullable Map<String, String> fields) throws IgniteObjectException {
         assert newMeta != null;
 
-        Map<String, String> oldFields = oldMeta != null ? ((PortableMetaDataImpl)oldMeta).fieldsMeta() : null;
-        Map<String, String> newFields = ((PortableMetaDataImpl)newMeta).fieldsMeta();
+        Map<String, String> oldFields = oldMeta != null ? ((IgniteObjectMetaDataImpl)oldMeta).fieldsMeta() : null;
+        Map<String, String> newFields = ((IgniteObjectMetaDataImpl)newMeta).fieldsMeta();
 
         boolean changed = false;
 
         if (oldMeta != null) {
             if (!oldMeta.typeName().equals(newMeta.typeName())) {
-                throw new PortableException(
+                throw new IgniteObjectException(
                     "Two portable types have duplicate type ID [" +
                         "typeId=" + typeId +
                         ", typeName1=" + oldMeta.typeName() +
@@ -851,7 +851,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
             }
 
             if (!F.eq(oldMeta.affinityKeyFieldName(), newMeta.affinityKeyFieldName())) {
-                throw new PortableException(
+                throw new IgniteObjectException(
                     "Portable type has different affinity key fields on different clients [" +
                         "typeName=" + newMeta.typeName() +
                         ", affKeyFieldName1=" + oldMeta.affinityKeyFieldName() +
@@ -871,7 +871,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
 
             if (typeName != null) {
                 if (!typeName.equals(e.getValue())) {
-                    throw new PortableException(
+                    throw new IgniteObjectException(
                         "Portable field has different types on different clients [" +
                             "typeName=" + newMeta.typeName() +
                             ", fieldName=" + e.getKey() +
@@ -895,7 +895,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     /**
      */
     private static class MetaDataProcessor implements
-        EntryProcessor<PortableMetaDataKey, PortableMetadata, PortableException>, Externalizable {
+        EntryProcessor<PortableMetaDataKey, IgniteObjectMetadata, IgniteObjectException>, Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -903,7 +903,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         private int typeId;
 
         /** */
-        private PortableMetadata newMeta;
+        private IgniteObjectMetadata newMeta;
 
         /**
          * For {@link Externalizable}.
@@ -916,7 +916,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
          * @param typeId Type ID.
          * @param newMeta New metadata.
          */
-        private MetaDataProcessor(int typeId, PortableMetadata newMeta) {
+        private MetaDataProcessor(int typeId, IgniteObjectMetadata newMeta) {
             assert newMeta != null;
 
             this.typeId = typeId;
@@ -924,16 +924,16 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         }
 
         /** {@inheritDoc} */
-        @Override public PortableException process(
-            MutableEntry<PortableMetaDataKey, PortableMetadata> entry,
+        @Override public IgniteObjectException process(
+            MutableEntry<PortableMetaDataKey, IgniteObjectMetadata> entry,
             Object... args) {
             try {
-                PortableMetadata oldMeta = entry.getValue();
+                IgniteObjectMetadata oldMeta = entry.getValue();
 
                 Map<String, String> fields = new HashMap<>();
 
                 if (checkMeta(typeId, oldMeta, newMeta, fields)) {
-                    PortableMetadata res = new PortableMetaDataImpl(newMeta.typeName(),
+                    IgniteObjectMetadata res = new IgniteObjectMetaDataImpl(newMeta.typeName(),
                         fields,
                         newMeta.affinityKeyFieldName());
 
@@ -944,7 +944,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
                 else
                     return null;
             }
-            catch (PortableException e) {
+            catch (IgniteObjectException e) {
                 return e;
             }
         }
@@ -958,7 +958,7 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             typeId = in.readInt();
-            newMeta = (PortableMetadata)in.readObject();
+            newMeta = (IgniteObjectMetadata)in.readObject();
         }
 
         /** {@inheritDoc} */
@@ -970,17 +970,17 @@ public class CacheObjectPortableProcessorImpl extends IgniteCacheObjectProcessor
     /**
      *
      */
-    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, PortableMetadata> {
+    class MetaDataEntryListener implements CacheEntryUpdatedListener<PortableMetaDataKey, IgniteObjectMetadata> {
         /** {@inheritDoc} */
         @Override public void onUpdated(
-            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends PortableMetadata>> evts)
+            Iterable<CacheEntryEvent<? extends PortableMetaDataKey, ? extends IgniteObjectMetadata>> evts)
             throws CacheEntryListenerException {
-            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends PortableMetadata> evt : evts) {
+            for (CacheEntryEvent<? extends PortableMetaDataKey, ? extends IgniteObjectMetadata> evt : evts) {
                 assert evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED : evt;
 
                 PortableMetaDataKey key = evt.getKey();
 
-                final PortableMetadata newMeta = evt.getValue();
+                final IgniteObjectMetadata newMeta = evt.getValue();
 
                 assert newMeta != null : evt;
 


[03/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractSelfTest.java
new file mode 100644
index 0000000..09f8527
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractSelfTest.java
@@ -0,0 +1,981 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import javax.cache.Cache;
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteObjects;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.portable.IgniteObjectImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.util.typedef.P2;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiInClosure;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public abstract class GridCacheIgniteObjectsAbstractSelfTest extends GridCommonAbstractTest {
+    /** */
+    public static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int ENTRY_CNT = 100;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheMode(cacheMode());
+        cacheCfg.setAtomicityMode(atomicityMode());
+        cacheCfg.setNearConfiguration(nearConfiguration());
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg.setCacheStoreFactory(singletonFactory(new TestStore()));
+        cacheCfg.setReadThrough(true);
+        cacheCfg.setWriteThrough(true);
+        cacheCfg.setLoadPreviousValue(true);
+        cacheCfg.setBackups(1);
+
+        if (offheapTiered()) {
+            cacheCfg.setMemoryMode(OFFHEAP_TIERED);
+            cacheCfg.setOffHeapMaxMemory(0);
+        }
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        cfg.setMarshaller(new PortableMarshaller());
+
+        return cfg;
+    }
+
+    /**
+     * @return {@code True} if should use OFFHEAP_TIERED mode.
+     */
+    protected boolean offheapTiered() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        for (int i = 0; i < gridCount(); i++) {
+            GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
+
+            for (GridCacheEntryEx e : c.map().entries0()) {
+                Object key = e.key().value(c.context().cacheObjectContext(), false);
+                Object val = CU.value(e.rawGet(), c.context(), false);
+
+                if (key instanceof IgniteObject)
+                    assert ((IgniteObjectImpl)key).detached() : val;
+
+                if (val instanceof IgniteObject)
+                    assert ((IgniteObjectImpl)val).detached() : val;
+            }
+        }
+
+        IgniteCache<Object, Object> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.remove(i);
+
+        if (offheapTiered()) {
+            for (int k = 0; k < 100; k++)
+                c.remove(k);
+        }
+
+        assertEquals(0, c.size());
+    }
+
+    /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode cacheMode();
+
+    /**
+     * @return Atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
+    /**
+     * @return Distribution mode.
+     */
+    protected abstract NearCacheConfiguration nearConfiguration();
+
+    /**
+     * @return Grid count.
+     */
+    protected abstract int gridCount();
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testCircularReference() throws Exception {
+        IgniteCache c = keepPortableCache();
+
+        TestReferenceObject obj1 = new TestReferenceObject();
+
+        obj1.obj = new TestReferenceObject(obj1);
+
+        c.put(1, obj1);
+
+        IgniteObject po = (IgniteObject)c.get(1);
+
+        String str = po.toString();
+
+        log.info("toString: " + str);
+
+        assertNotNull(str);
+
+        assertTrue("Unexpected toString: " + str,
+            str.startsWith("TestReferenceObject") && str.contains("obj=TestReferenceObject ["));
+
+        TestReferenceObject obj1_r = po.deserialize();
+
+        assertNotNull(obj1_r);
+
+        TestReferenceObject obj2_r = obj1_r.obj;
+
+        assertNotNull(obj2_r);
+
+        assertSame(obj1_r, obj2_r.obj);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGet() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject obj = c.get(i);
+
+            assertEquals(i, obj.val);
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            IgniteObject po = kpc.get(i);
+
+            assertEquals(i, (int)po.field("val"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIterator() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        Map<Integer, TestObject> entries = new HashMap<>();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            TestObject val = new TestObject(i);
+
+            c.put(i, val);
+
+            entries.put(i, val);
+        }
+
+        IgniteCache<Integer, IgniteObject> prj = ((IgniteCacheProxy)c).keepPortable();
+
+        Iterator<Cache.Entry<Integer, IgniteObject>> it = prj.iterator();
+
+        assertTrue(it.hasNext());
+
+        while (it.hasNext()) {
+            Cache.Entry<Integer, IgniteObject> entry = it.next();
+
+            assertTrue(entries.containsKey(entry.getKey()));
+
+            TestObject o = entries.get(entry.getKey());
+
+            IgniteObject po = entry.getValue();
+
+            assertEquals(o.val, (int)po.field("val"));
+
+            entries.remove(entry.getKey());
+        }
+
+        assertEquals(0, entries.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollection() throws Exception {
+        IgniteCache<Integer, Collection<TestObject>> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Collection<TestObject> col = new ArrayList<>(3);
+
+            for (int j = 0; j < 3; j++)
+                col.add(new TestObject(i * 10 + j));
+
+            c.put(i, col);
+        }
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Collection<TestObject> col = c.get(i);
+
+            assertEquals(3, col.size());
+
+            Iterator<TestObject> it = col.iterator();
+
+            for (int j = 0; j < 3; j++) {
+                assertTrue(it.hasNext());
+
+                assertEquals(i * 10 + j, it.next().val);
+            }
+        }
+
+        IgniteCache<Integer, Collection<IgniteObject>> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Collection<IgniteObject> col = kpc.get(i);
+
+            assertEquals(3, col.size());
+
+            Iterator<IgniteObject> it = col.iterator();
+
+            for (int j = 0; j < 3; j++) {
+                assertTrue(it.hasNext());
+
+                assertEquals(i * 10 + j, (int)it.next().field("val"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMap() throws Exception {
+        IgniteCache<Integer, Map<Integer, TestObject>> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Map<Integer, TestObject> map = U.newHashMap(3);
+
+            for (int j = 0; j < 3; j++) {
+                int idx = i * 10 + j;
+
+                map.put(idx, new TestObject(idx));
+            }
+
+            c.put(i, map);
+        }
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Map<Integer, TestObject> map = c.get(i);
+
+            assertEquals(3, map.size());
+
+            for (int j = 0; j < 3; j++) {
+                int idx = i * 10 + j;
+
+                assertEquals(idx, map.get(idx).val);
+            }
+        }
+
+        IgniteCache<Integer, Map<Integer, IgniteObject>> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            Map<Integer, IgniteObject> map = kpc.get(i);
+
+            assertEquals(3, map.size());
+
+            for (int j = 0; j < 3; j++) {
+                int idx = i * 10 + j;
+
+                assertEquals(idx, (int)map.get(idx).field("val"));
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAsync() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            cacheAsync.get(i);
+
+            TestObject obj = cacheAsync.<TestObject>future().get();
+
+            assertNotNull(obj);
+
+            assertEquals(i, obj.val);
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+
+        IgniteCache<Integer, IgniteObject> cachePortableAsync = kpc.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            cachePortableAsync.get(i);
+
+            IgniteObject po = cachePortableAsync.<IgniteObject>future().get();
+
+            assertEquals(i, (int)po.field("val"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetTx() throws Exception {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                TestObject obj = c.get(i);
+
+                assertEquals(i, obj.val);
+
+                tx.commit();
+            }
+        }
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                TestObject obj = c.get(i);
+
+                assertEquals(i, obj.val);
+
+                tx.commit();
+            }
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                IgniteObject po = kpc.get(i);
+
+                assertEquals(i, (int)po.field("val"));
+
+                tx.commit();
+            }
+        }
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                IgniteObject po = kpc.get(i);
+
+                assertEquals(i, (int)po.field("val"));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAsyncTx() throws Exception {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cacheAsync.get(i);
+
+                TestObject obj = cacheAsync.<TestObject>future().get();
+
+                assertEquals(i, obj.val);
+
+                tx.commit();
+            }
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+        IgniteCache<Integer, IgniteObject> cachePortableAsync = kpc.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++) {
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cachePortableAsync.get(i);
+
+                IgniteObject po = cachePortableAsync.<IgniteObject>future().get();
+
+                assertEquals(i, (int)po.field("val"));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAll() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            Map<Integer, TestObject> objs = c.getAll(keys);
+
+            assertEquals(10, objs.size());
+
+            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
+                assertEquals(e.getKey().intValue(), e.getValue().val);
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            Map<Integer, IgniteObject> objs = kpc.getAll(keys);
+
+            assertEquals(10, objs.size());
+
+            for (Map.Entry<Integer, IgniteObject> e : objs.entrySet())
+                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllAsync() throws Exception {
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            cacheAsync.getAll(keys);
+
+            Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
+
+            assertEquals(10, objs.size());
+
+            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
+                assertEquals(e.getKey().intValue(), e.getValue().val);
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+        IgniteCache<Integer, IgniteObject> cachePortableAsync = kpc.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+
+            cachePortableAsync.getAll(keys);
+
+            Map<Integer, IgniteObject> objs = cachePortableAsync.<Map<Integer, IgniteObject>>future().get();
+
+            assertEquals(10, objs.size());
+
+            for (Map.Entry<Integer, IgniteObject> e : objs.entrySet())
+                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllTx() throws Exception {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        IgniteCache<Integer, TestObject> c = jcache(0);
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Map<Integer, TestObject> objs = c.getAll(keys);
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
+                    assertEquals(e.getKey().intValue(), e.getValue().val);
+
+                tx.commit();
+            }
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                Map<Integer, TestObject> objs = c.getAll(keys);
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
+                    assertEquals(e.getKey().intValue(), e.getValue().val);
+
+                tx.commit();
+            }
+        }
+
+        IgniteCache<Integer, IgniteObject> kpc = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Map<Integer, IgniteObject> objs = kpc.getAll(keys);
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, IgniteObject> e : objs.entrySet())
+                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
+
+                tx.commit();
+            }
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
+                Map<Integer, IgniteObject> objs = kpc.getAll(keys);
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, IgniteObject> e : objs.entrySet())
+                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetAllAsyncTx() throws Exception {
+        if (atomicityMode() != TRANSACTIONAL)
+            return;
+
+        IgniteCache<Integer, TestObject> c = jcache(0);
+        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
+
+        for (int i = 0; i < ENTRY_CNT; i++)
+            c.put(i, new TestObject(i));
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                cacheAsync.getAll(keys);
+
+                Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
+                    assertEquals(e.getKey().intValue(), e.getValue().val);
+
+                tx.commit();
+            }
+        }
+
+        IgniteCache<Integer, IgniteObject> cache = keepPortableCache();
+
+        for (int i = 0; i < ENTRY_CNT; ) {
+            Set<Integer> keys = new HashSet<>();
+
+            for (int j = 0; j < 10; j++)
+                keys.add(i++);
+
+            IgniteCache<Integer, IgniteObject> asyncCache = cache.withAsync();
+
+            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                asyncCache.getAll(keys);
+
+                Map<Integer, IgniteObject> objs = asyncCache.<Map<Integer, IgniteObject>>future().get();
+
+                assertEquals(10, objs.size());
+
+                for (Map.Entry<Integer, IgniteObject> e : objs.entrySet())
+                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
+
+                tx.commit();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCache() throws Exception {
+        for (int i = 0; i < gridCount(); i++)
+            jcache(i).localLoadCache(null);
+
+        IgniteCache<Integer, TestObject> cache = jcache(0);
+
+        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
+
+        assertEquals(1, cache.get(1).val);
+        assertEquals(2, cache.get(2).val);
+        assertEquals(3, cache.get(3).val);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheAsync() throws Exception {
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteCache<Object, Object> jcache = jcache(i).withAsync();
+
+            jcache.loadCache(null);
+
+            jcache.future().get();
+        }
+
+        IgniteCache<Integer, TestObject> cache = jcache(0);
+
+        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
+
+        assertEquals(1, cache.get(1).val);
+        assertEquals(2, cache.get(2).val);
+        assertEquals(3, cache.get(3).val);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLoadCacheFilteredAsync() throws Exception {
+        for (int i = 0; i < gridCount(); i++) {
+            IgniteCache<Integer, TestObject> c = this.<Integer, TestObject>jcache(i).withAsync();
+
+            c.loadCache(new P2<Integer, TestObject>() {
+                @Override public boolean apply(Integer key, TestObject val) {
+                    return val.val < 3;
+                }
+            });
+
+            c.future().get();
+        }
+
+        IgniteCache<Integer, TestObject> cache = jcache(0);
+
+        assertEquals(2, cache.size(CachePeekMode.PRIMARY));
+
+        assertEquals(1, cache.get(1).val);
+        assertEquals(2, cache.get(2).val);
+
+        assertNull(cache.get(3));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTransform() throws Exception {
+        IgniteCache<Integer, IgniteObject> c = keepPortableCache();
+
+        checkTransform(primaryKey(c));
+
+        if (cacheMode() != CacheMode.LOCAL) {
+            checkTransform(backupKey(c));
+
+            if (nearConfiguration() != null)
+                checkTransform(nearKey(c));
+        }
+    }
+
+    /**
+     * @return Cache with keep portable flag.
+     */
+    private <K, V> IgniteCache<K, V> keepPortableCache() {
+        return ignite(0).cache(null).withKeepBinary();
+    }
+
+    /**
+     * @param key Key.
+     * @throws Exception If failed.
+     */
+    private void checkTransform(Integer key) throws Exception {
+        log.info("Transform: " + key);
+
+        IgniteCache<Integer, IgniteObject> c = keepPortableCache();
+
+        try {
+            c.invoke(key, new EntryProcessor<Integer, IgniteObject, Void>() {
+                @Override public Void process(MutableEntry<Integer, IgniteObject> e, Object... args) {
+                    IgniteObject val = e.getValue();
+
+                    assertNull("Unexpected value: " + val, val);
+
+                    return null;
+                }
+            });
+
+            jcache(0).put(key, new TestObject(1));
+
+            c.invoke(key, new EntryProcessor<Integer, IgniteObject, Void>() {
+                @Override public Void process(MutableEntry<Integer, IgniteObject> e, Object... args) {
+                    IgniteObject val = e.getValue();
+
+                    assertNotNull("Unexpected value: " + val, val);
+
+                    assertEquals(new Integer(1), val.field("val"));
+
+                    Ignite ignite = e.unwrap(Ignite.class);
+
+                    IgniteObjects portables = ignite.portables();
+
+                    IgniteObjectBuilder builder = portables.builder(val);
+
+                    builder.setField("val", 2);
+
+                    e.setValue(builder.build());
+
+                    return null;
+                }
+            });
+
+            IgniteObject obj = c.get(key);
+
+            assertEquals(new Integer(2), obj.field("val"));
+
+            c.invoke(key, new EntryProcessor<Integer, IgniteObject, Void>() {
+                @Override public Void process(MutableEntry<Integer, IgniteObject> e, Object... args) {
+                    IgniteObject val = e.getValue();
+
+                    assertNotNull("Unexpected value: " + val, val);
+
+                    assertEquals(new Integer(2), val.field("val"));
+
+                    e.setValue(val);
+
+                    return null;
+                }
+            });
+
+            obj = c.get(key);
+
+            assertEquals(new Integer(2), obj.field("val"));
+
+            c.invoke(key, new EntryProcessor<Integer, IgniteObject, Void>() {
+                @Override public Void process(MutableEntry<Integer, IgniteObject> e, Object... args) {
+                    IgniteObject val = e.getValue();
+
+                    assertNotNull("Unexpected value: " + val, val);
+
+                    assertEquals(new Integer(2), val.field("val"));
+
+                    e.remove();
+
+                    return null;
+                }
+            });
+
+            assertNull(c.get(key));
+        }
+        finally {
+            c.remove(key);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestObject implements IgniteObjectMarshalAware {
+        /** */
+        private int val;
+
+        /**
+         */
+        private TestObject() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        private TestObject(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+            writer.writeInt("val", val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+            val = reader.readInt("val");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestReferenceObject implements IgniteObjectMarshalAware {
+        /** */
+        private TestReferenceObject obj;
+
+        /**
+         */
+        private TestReferenceObject() {
+            // No-op.
+        }
+
+        /**
+         * @param obj Object.
+         */
+        private TestReferenceObject(TestReferenceObject obj) {
+            this.obj = obj;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+            writer.writeObject("obj", obj);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+            obj = reader.readObject("obj");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestStore extends CacheStoreAdapter<Integer, Object> {
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Object> clo, Object... args) {
+            for (int i = 1; i <= 3; i++)
+                clo.apply(i, new TestObject(i));
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object load(Integer key) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ?> e) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
deleted file mode 100644
index 9ba38d9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractDataStreamerSelfTest.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractDataStreamerSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int THREAD_CNT = 64;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject.class.getName())));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /**
-     * @return Sync mode.
-     */
-    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return PRIMARY_SYNC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Near configuration.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected int gridCount() {
-        return 1;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    public void testGetPut() throws Exception {
-        final AtomicBoolean flag = new AtomicBoolean();
-
-        final LongAdder8 cnt = new LongAdder8();
-
-        try (IgniteDataStreamer<Object, Object> ldr = grid(0).dataStreamer(null)) {
-            IgniteInternalFuture<?> f = multithreadedAsync(
-                new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                        while (!flag.get()) {
-                            ldr.addData(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
-
-                            cnt.add(1);
-                        }
-
-                        return null;
-                    }
-                },
-                THREAD_CNT
-            );
-
-            for (int i = 0; i < 30 && !f.isDone(); i++)
-                Thread.sleep(1000);
-
-            flag.set(true);
-
-            f.get();
-        }
-
-        info("Operations in 30 sec: " + cnt.sum());
-    }
-
-    /**
-     */
-    private static class TestObject implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject && ((TestObject)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
deleted file mode 100644
index 67f0e52..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractMultiThreadedSelfTest.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractMultiThreadedSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final int THREAD_CNT = 64;
-
-    /** */
-    private static final AtomicInteger idxGen = new AtomicInteger();
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject.class.getName())));
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /**
-     * @return Sync mode.
-     */
-    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
-        return PRIMARY_SYNC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Distribution mode.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected int gridCount() {
-        return 1;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("BusyWait") public void testGetPut() throws Exception {
-        final AtomicBoolean flag = new AtomicBoolean();
-
-        final LongAdder8 cnt = new LongAdder8();
-
-        IgniteInternalFuture<?> f = multithreadedAsync(
-            new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    int threadId = idxGen.getAndIncrement() % 2;
-
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    while (!flag.get()) {
-                        IgniteCache<Object, Object> c = jcache(rnd.nextInt(gridCount()));
-
-                        switch (threadId) {
-                            case 0:
-                                // Put/get/remove portable -> portable.
-
-                                c.put(new TestObject(rnd.nextInt(10000)), new TestObject(rnd.nextInt(10000)));
-
-                                IgniteCache<Object, Object> p2 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
-
-                                PortableObject v = (PortableObject)p2.get(new TestObject(rnd.nextInt(10000)));
-
-                                if (v != null)
-                                    v.deserialize();
-
-                                c.remove(new TestObject(rnd.nextInt(10000)));
-
-                                break;
-
-                            case 1:
-                                // Put/get int -> portable.
-                                c.put(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
-
-                                IgniteCache<Integer, PortableObject> p4 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
-
-                                PortableObject v1 = p4.get(rnd.nextInt(10000));
-
-                                if (v1 != null)
-                                    v1.deserialize();
-
-                                p4.remove(rnd.nextInt(10000));
-
-                                break;
-
-                            default:
-                                assert false;
-                        }
-
-                        cnt.add(3);
-                    }
-
-                    return null;
-                }
-            },
-            THREAD_CNT
-        );
-
-        for (int i = 0; i < 30 && !f.isDone(); i++)
-            Thread.sleep(1000);
-
-        flag.set(true);
-
-        f.get();
-
-        info("Operations in 30 sec: " + cnt.sum());
-    }
-
-    /**
-     */
-    private static class TestObject implements PortableMarshalAware, Serializable {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj instanceof TestObject && ((TestObject)obj).val == val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
deleted file mode 100644
index 5213d21..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableObjectsAbstractSelfTest.java
+++ /dev/null
@@ -1,981 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import javax.cache.Cache;
-import javax.cache.processor.EntryProcessor;
-import javax.cache.processor.MutableEntry;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CachePeekMode;
-import org.apache.ignite.cache.store.CacheStoreAdapter;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.portable.PortableObjectImpl;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.util.typedef.P2;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lang.IgniteBiInClosure;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.transactions.Transaction;
-import org.jetbrains.annotations.Nullable;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheMemoryMode.OFFHEAP_TIERED;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
-import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
-import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
-
-/**
- * Test for portable objects stored in cache.
- */
-public abstract class GridCachePortableObjectsAbstractSelfTest extends GridCommonAbstractTest {
-    /** */
-    public static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final int ENTRY_CNT = 100;
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disco);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(cacheMode());
-        cacheCfg.setAtomicityMode(atomicityMode());
-        cacheCfg.setNearConfiguration(nearConfiguration());
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg.setCacheStoreFactory(singletonFactory(new TestStore()));
-        cacheCfg.setReadThrough(true);
-        cacheCfg.setWriteThrough(true);
-        cacheCfg.setLoadPreviousValue(true);
-        cacheCfg.setBackups(1);
-
-        if (offheapTiered()) {
-            cacheCfg.setMemoryMode(OFFHEAP_TIERED);
-            cacheCfg.setOffHeapMaxMemory(0);
-        }
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        return cfg;
-    }
-
-    /**
-     * @return {@code True} if should use OFFHEAP_TIERED mode.
-     */
-    protected boolean offheapTiered() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGridsMultiThreaded(gridCount());
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
-
-            for (GridCacheEntryEx e : c.map().entries0()) {
-                Object key = e.key().value(c.context().cacheObjectContext(), false);
-                Object val = CU.value(e.rawGet(), c.context(), false);
-
-                if (key instanceof PortableObject)
-                    assert ((PortableObjectImpl)key).detached() : val;
-
-                if (val instanceof PortableObject)
-                    assert ((PortableObjectImpl)val).detached() : val;
-            }
-        }
-
-        IgniteCache<Object, Object> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.remove(i);
-
-        if (offheapTiered()) {
-            for (int k = 0; k < 100; k++)
-                c.remove(k);
-        }
-
-        assertEquals(0, c.size());
-    }
-
-    /**
-     * @return Cache mode.
-     */
-    protected abstract CacheMode cacheMode();
-
-    /**
-     * @return Atomicity mode.
-     */
-    protected abstract CacheAtomicityMode atomicityMode();
-
-    /**
-     * @return Distribution mode.
-     */
-    protected abstract NearCacheConfiguration nearConfiguration();
-
-    /**
-     * @return Grid count.
-     */
-    protected abstract int gridCount();
-
-    /**
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public void testCircularReference() throws Exception {
-        IgniteCache c = keepPortableCache();
-
-        TestReferenceObject obj1 = new TestReferenceObject();
-
-        obj1.obj = new TestReferenceObject(obj1);
-
-        c.put(1, obj1);
-
-        PortableObject po = (PortableObject)c.get(1);
-
-        String str = po.toString();
-
-        log.info("toString: " + str);
-
-        assertNotNull(str);
-
-        assertTrue("Unexpected toString: " + str,
-            str.startsWith("TestReferenceObject") && str.contains("obj=TestReferenceObject ["));
-
-        TestReferenceObject obj1_r = po.deserialize();
-
-        assertNotNull(obj1_r);
-
-        TestReferenceObject obj2_r = obj1_r.obj;
-
-        assertNotNull(obj2_r);
-
-        assertSame(obj1_r, obj2_r.obj);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGet() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            TestObject obj = c.get(i);
-
-            assertEquals(i, obj.val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            PortableObject po = kpc.get(i);
-
-            assertEquals(i, (int)po.field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIterator() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        Map<Integer, TestObject> entries = new HashMap<>();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            TestObject val = new TestObject(i);
-
-            c.put(i, val);
-
-            entries.put(i, val);
-        }
-
-        IgniteCache<Integer, PortableObject> prj = ((IgniteCacheProxy)c).keepPortable();
-
-        Iterator<Cache.Entry<Integer, PortableObject>> it = prj.iterator();
-
-        assertTrue(it.hasNext());
-
-        while (it.hasNext()) {
-            Cache.Entry<Integer, PortableObject> entry = it.next();
-
-            assertTrue(entries.containsKey(entry.getKey()));
-
-            TestObject o = entries.get(entry.getKey());
-
-            PortableObject po = entry.getValue();
-
-            assertEquals(o.val, (int)po.field("val"));
-
-            entries.remove(entry.getKey());
-        }
-
-        assertEquals(0, entries.size());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollection() throws Exception {
-        IgniteCache<Integer, Collection<TestObject>> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<TestObject> col = new ArrayList<>(3);
-
-            for (int j = 0; j < 3; j++)
-                col.add(new TestObject(i * 10 + j));
-
-            c.put(i, col);
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<TestObject> col = c.get(i);
-
-            assertEquals(3, col.size());
-
-            Iterator<TestObject> it = col.iterator();
-
-            for (int j = 0; j < 3; j++) {
-                assertTrue(it.hasNext());
-
-                assertEquals(i * 10 + j, it.next().val);
-            }
-        }
-
-        IgniteCache<Integer, Collection<PortableObject>> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Collection<PortableObject> col = kpc.get(i);
-
-            assertEquals(3, col.size());
-
-            Iterator<PortableObject> it = col.iterator();
-
-            for (int j = 0; j < 3; j++) {
-                assertTrue(it.hasNext());
-
-                assertEquals(i * 10 + j, (int)it.next().field("val"));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMap() throws Exception {
-        IgniteCache<Integer, Map<Integer, TestObject>> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, TestObject> map = U.newHashMap(3);
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                map.put(idx, new TestObject(idx));
-            }
-
-            c.put(i, map);
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, TestObject> map = c.get(i);
-
-            assertEquals(3, map.size());
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                assertEquals(idx, map.get(idx).val);
-            }
-        }
-
-        IgniteCache<Integer, Map<Integer, PortableObject>> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            Map<Integer, PortableObject> map = kpc.get(i);
-
-            assertEquals(3, map.size());
-
-            for (int j = 0; j < 3; j++) {
-                int idx = i * 10 + j;
-
-                assertEquals(idx, (int)map.get(idx).field("val"));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAsync() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            cacheAsync.get(i);
-
-            TestObject obj = cacheAsync.<TestObject>future().get();
-
-            assertNotNull(obj);
-
-            assertEquals(i, obj.val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            cachePortableAsync.get(i);
-
-            PortableObject po = cachePortableAsync.<PortableObject>future().get();
-
-            assertEquals(i, (int)po.field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                TestObject obj = c.get(i);
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                TestObject obj = c.get(i);
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                PortableObject po = kpc.get(i);
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                PortableObject po = kpc.get(i);
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAsyncTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cacheAsync.get(i);
-
-                TestObject obj = cacheAsync.<TestObject>future().get();
-
-                assertEquals(i, obj.val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++) {
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cachePortableAsync.get(i);
-
-                PortableObject po = cachePortableAsync.<PortableObject>future().get();
-
-                assertEquals(i, (int)po.field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAll() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            Map<Integer, TestObject> objs = c.getAll(keys);
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                assertEquals(e.getKey().intValue(), e.getValue().val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllAsync() throws Exception {
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            cacheAsync.getAll(keys);
-
-            Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                assertEquals(e.getKey().intValue(), e.getValue().val);
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-        IgniteCache<Integer, PortableObject> cachePortableAsync = kpc.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-
-            cachePortableAsync.getAll(keys);
-
-            Map<Integer, PortableObject> objs = cachePortableAsync.<Map<Integer, PortableObject>>future().get();
-
-            assertEquals(10, objs.size());
-
-            for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                Map<Integer, TestObject> objs = c.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                Map<Integer, TestObject> objs = c.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> kpc = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, READ_COMMITTED)) {
-                Map<Integer, PortableObject> objs = kpc.getAll(keys);
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetAllAsyncTx() throws Exception {
-        if (atomicityMode() != TRANSACTIONAL)
-            return;
-
-        IgniteCache<Integer, TestObject> c = jcache(0);
-        IgniteCache<Integer, TestObject> cacheAsync = c.withAsync();
-
-        for (int i = 0; i < ENTRY_CNT; i++)
-            c.put(i, new TestObject(i));
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                cacheAsync.getAll(keys);
-
-                Map<Integer, TestObject> objs = cacheAsync.<Map<Integer, TestObject>>future().get();
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, TestObject> e : objs.entrySet())
-                    assertEquals(e.getKey().intValue(), e.getValue().val);
-
-                tx.commit();
-            }
-        }
-
-        IgniteCache<Integer, PortableObject> cache = keepPortableCache();
-
-        for (int i = 0; i < ENTRY_CNT; ) {
-            Set<Integer> keys = new HashSet<>();
-
-            for (int j = 0; j < 10; j++)
-                keys.add(i++);
-
-            IgniteCache<Integer, PortableObject> asyncCache = cache.withAsync();
-
-            try (Transaction tx = grid(0).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                asyncCache.getAll(keys);
-
-                Map<Integer, PortableObject> objs = asyncCache.<Map<Integer, PortableObject>>future().get();
-
-                assertEquals(10, objs.size());
-
-                for (Map.Entry<Integer, PortableObject> e : objs.entrySet())
-                    assertEquals(new Integer(e.getKey().intValue()), e.getValue().field("val"));
-
-                tx.commit();
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCache() throws Exception {
-        for (int i = 0; i < gridCount(); i++)
-            jcache(i).localLoadCache(null);
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-        assertEquals(3, cache.get(3).val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheAsync() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            IgniteCache<Object, Object> jcache = jcache(i).withAsync();
-
-            jcache.loadCache(null);
-
-            jcache.future().get();
-        }
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(3, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-        assertEquals(3, cache.get(3).val);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLoadCacheFilteredAsync() throws Exception {
-        for (int i = 0; i < gridCount(); i++) {
-            IgniteCache<Integer, TestObject> c = this.<Integer, TestObject>jcache(i).withAsync();
-
-            c.loadCache(new P2<Integer, TestObject>() {
-                @Override public boolean apply(Integer key, TestObject val) {
-                    return val.val < 3;
-                }
-            });
-
-            c.future().get();
-        }
-
-        IgniteCache<Integer, TestObject> cache = jcache(0);
-
-        assertEquals(2, cache.size(CachePeekMode.PRIMARY));
-
-        assertEquals(1, cache.get(1).val);
-        assertEquals(2, cache.get(2).val);
-
-        assertNull(cache.get(3));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTransform() throws Exception {
-        IgniteCache<Integer, PortableObject> c = keepPortableCache();
-
-        checkTransform(primaryKey(c));
-
-        if (cacheMode() != CacheMode.LOCAL) {
-            checkTransform(backupKey(c));
-
-            if (nearConfiguration() != null)
-                checkTransform(nearKey(c));
-        }
-    }
-
-    /**
-     * @return Cache with keep portable flag.
-     */
-    private <K, V> IgniteCache<K, V> keepPortableCache() {
-        return ignite(0).cache(null).withKeepPortable();
-    }
-
-    /**
-     * @param key Key.
-     * @throws Exception If failed.
-     */
-    private void checkTransform(Integer key) throws Exception {
-        log.info("Transform: " + key);
-
-        IgniteCache<Integer, PortableObject> c = keepPortableCache();
-
-        try {
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNull("Unexpected value: " + val, val);
-
-                    return null;
-                }
-            });
-
-            jcache(0).put(key, new TestObject(1));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(1), val.field("val"));
-
-                    Ignite ignite = e.unwrap(Ignite.class);
-
-                    IgnitePortables portables = ignite.portables();
-
-                    PortableBuilder builder = portables.builder(val);
-
-                    builder.setField("val", 2);
-
-                    e.setValue(builder.build());
-
-                    return null;
-                }
-            });
-
-            PortableObject obj = c.get(key);
-
-            assertEquals(new Integer(2), obj.field("val"));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(2), val.field("val"));
-
-                    e.setValue(val);
-
-                    return null;
-                }
-            });
-
-            obj = c.get(key);
-
-            assertEquals(new Integer(2), obj.field("val"));
-
-            c.invoke(key, new EntryProcessor<Integer, PortableObject, Void>() {
-                @Override public Void process(MutableEntry<Integer, PortableObject> e, Object... args) {
-                    PortableObject val = e.getValue();
-
-                    assertNotNull("Unexpected value: " + val, val);
-
-                    assertEquals(new Integer(2), val.field("val"));
-
-                    e.remove();
-
-                    return null;
-                }
-            });
-
-            assertNull(c.get(key));
-        }
-        finally {
-            c.remove(key);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestObject implements PortableMarshalAware {
-        /** */
-        private int val;
-
-        /**
-         */
-        private TestObject() {
-            // No-op.
-        }
-
-        /**
-         * @param val Value.
-         */
-        private TestObject(int val) {
-            this.val = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeInt("val", val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            val = reader.readInt("val");
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestReferenceObject implements PortableMarshalAware {
-        /** */
-        private TestReferenceObject obj;
-
-        /**
-         */
-        private TestReferenceObject() {
-            // No-op.
-        }
-
-        /**
-         * @param obj Object.
-         */
-        private TestReferenceObject(TestReferenceObject obj) {
-            this.obj = obj;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            writer.writeObject("obj", obj);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            obj = reader.readObject("obj");
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestStore extends CacheStoreAdapter<Integer, Object> {
-        /** {@inheritDoc} */
-        @Override public void loadCache(IgniteBiInClosure<Integer, Object> clo, Object... args) {
-            for (int i = 1; i <= 3; i++)
-                clo.apply(i, new TestObject(i));
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object load(Integer key) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Cache.Entry<? extends Integer, ?> e) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void delete(Object key) {
-            // No-op.
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
index 5c0fc8e..3a7f8fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCachePortableStorePortablesSelfTest.java
@@ -17,7 +17,7 @@
 package org.apache.ignite.internal.processors.cache.portable;
 
 import java.util.Map;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * Tests for cache store with portables.
@@ -47,9 +47,9 @@ public class GridCachePortableStorePortablesSelfTest extends GridCachePortableSt
         for (int idx : idxs) {
             Object val = map.get(portable(new Key(idx)));
 
-            assertTrue(String.valueOf(val), val instanceof PortableObject);
+            assertTrue(String.valueOf(val), val instanceof IgniteObject);
 
-            PortableObject po = (PortableObject)val;
+            IgniteObject po = (IgniteObject)val;
 
             assertEquals("Value", po.metaData().typeName());
             assertEquals(new Integer(idx), po.field("idx"));

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
index 0db650e..5799017 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableCacheEntryMemorySizeSelfTest.java
@@ -25,8 +25,8 @@ import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
 
 /**
  *
@@ -39,11 +39,11 @@ public class GridPortableCacheEntryMemorySizeSelfTest extends GridCacheEntryMemo
         marsh.setContext(new MarshallerContextTestImpl(null));
 
         PortableContext pCtx = new PortableContext(new PortableMetaDataHandler() {
-            @Override public void addMeta(int typeId, PortableMetadata meta) throws PortableException {
+            @Override public void addMeta(int typeId, IgniteObjectMetadata meta) throws IgniteObjectException {
                 // No-op
             }
 
-            @Override public PortableMetadata metadata(int typeId) throws PortableException {
+            @Override public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException {
                 return null;
             }
         }, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
index a1a623b..1ba8136 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridPortableDuplicateIndexObjectsAbstractSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  * Tests that portable object is the same in cache entry and in index.
@@ -96,9 +96,9 @@ public abstract class GridPortableDuplicateIndexObjectsAbstractSelfTest extends
 
         cache.put(key, new TestPortable(fieldOneVal, fieldTwoVal));
 
-        IgniteCache<Integer, PortableObject> prj = grid(0).cache(null).withKeepPortable();
+        IgniteCache<Integer, IgniteObject> prj = grid(0).cache(null).withKeepBinary();
 
-        PortableObject cacheVal = prj.get(key);
+        IgniteObject cacheVal = prj.get(key);
 
         assertEquals(fieldOneVal, cacheVal.field("fieldOne"));
         assertEquals(new Integer(fieldTwoVal), cacheVal.field("fieldTwo"));
@@ -108,7 +108,7 @@ public abstract class GridPortableDuplicateIndexObjectsAbstractSelfTest extends
 
         assertEquals(1, row.size());
 
-        PortableObject qryVal = (PortableObject)row.get(0);
+        IgniteObject qryVal = (IgniteObject)row.get(0);
 
         assertEquals(fieldOneVal, qryVal.field("fieldOne"));
         assertEquals(new Integer(fieldTwoVal), qryVal.field("fieldTwo"));

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
index 836440a..ad21a47 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/datastreaming/DataStreamProcessorPortableSelfTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 import org.apache.ignite.stream.StreamReceiver;
 
 /**
@@ -55,9 +55,9 @@ public class DataStreamProcessorPortableSelfTest extends DataStreamProcessorSelf
             Collection<Map.Entry<String, TestObject>> entries) {
             for (Map.Entry<String, TestObject> e : entries) {
                 assertTrue(e.getKey() instanceof String);
-                assertTrue(e.getValue() instanceof PortableObject);
+                assertTrue(e.getValue() instanceof IgniteObject);
 
-                TestObject obj = ((PortableObject)e.getValue()).deserialize();
+                TestObject obj = ((IgniteObject)e.getValue()).deserialize();
 
                 cache.put(e.getKey(), new TestObject(obj.val + 1));
             }


[05/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
index 2b29e4d..d3d40bd 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerSelfTest.java
@@ -41,7 +41,7 @@ import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
+import org.apache.ignite.internal.portable.builder.IgniteObjectBuilderImpl;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
@@ -50,19 +50,19 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectRawReader;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectSerializer;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jsr166.ConcurrentHashMap8;
@@ -84,11 +84,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
     /** */
     protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
+        @Override public void addMeta(int typeId, IgniteObjectMetadata meta) {
             // No-op.
         }
 
-        @Override public PortableMetadata metadata(int typeId) {
+        @Override public IgniteObjectMetadata metadata(int typeId) {
             return null;
         }
     };
@@ -397,13 +397,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testPortableObject() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
+        marsh.setTypeConfigurations(Arrays.asList(new IgniteObjectConfiguration(SimpleObject.class.getName())));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject po0 = marshalUnmarshal(po, marsh);
+        IgniteObject po0 = marshalUnmarshal(po, marsh);
 
         assertTrue(po.hasField("b"));
         assertTrue(po.hasField("s"));
@@ -435,7 +435,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDateAndTimestampInSingleObject() throws Exception {
-        PortableTypeConfiguration cfg1 = new PortableTypeConfiguration(DateClass1.class.getName());
+        IgniteObjectConfiguration cfg1 = new IgniteObjectConfiguration(DateClass1.class.getName());
 
         PortableMarshaller marsh = new PortableMarshaller();
 
@@ -448,7 +448,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         obj1.date = date;
         obj1.ts = ts;
 
-        PortableObject po1 = marshal(obj1, marsh);
+        IgniteObject po1 = marshal(obj1, marsh);
 
         assertEquals(date, po1.field("date"));
         assertEquals(Date.class, po1.field("date").getClass());
@@ -467,12 +467,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         assertEquals(obj.hashCode(), po.hashCode());
 
@@ -509,7 +509,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("enumArr")));
         assertNull(po.field("unknown"));
 
-        PortableObject innerPo = po.field("inner");
+        IgniteObject innerPo = po.field("inner");
 
         assertEquals(obj.inner, innerPo.deserialize());
 
@@ -554,13 +554,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
-            new PortableTypeConfiguration(TestPortableObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName()),
+            new IgniteObjectConfiguration(TestIgniteObjectObject.class.getName())
         ));
 
-        TestPortableObject obj = portableObject();
+        TestIgniteObjectObject obj = portableObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         assertEquals(obj.hashCode(), po.hashCode());
 
@@ -596,7 +596,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertArrayEquals(ordinals(obj.enumArr), ordinals((Enum<?>[])po.field("_enumArr")));
         assertNull(po.field("unknown"));
 
-        PortableObject simplePo = po.field("_simple");
+        IgniteObject simplePo = po.field("_simple");
 
         assertEquals(obj.simple, simplePo.deserialize());
 
@@ -634,7 +634,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertNull(simplePo.field("portable"));
         assertNull(simplePo.field("unknown"));
 
-        PortableObject portablePo = po.field("_portable");
+        IgniteObject portablePo = po.field("_portable");
 
         assertEquals(obj.portable, portablePo.deserialize());
 
@@ -676,7 +676,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param obj Simple object.
      * @param po Portable object.
      */
-    private void checkSimpleObjectData(SimpleObject obj, PortableObject po) {
+    private void checkSimpleObjectData(SimpleObject obj, IgniteObject po) {
         assertEquals(obj.b, (byte)po.field("b"));
         assertEquals(obj.s, (short)po.field("s"));
         assertEquals(obj.i, (int)po.field("i"));
@@ -722,7 +722,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         U.intToBytes(Integer.reverseBytes(11111), arr, 3);
 
-        final PortableObject po = new PortableObjectImpl(initPortableContext(new PortableMarshaller()), arr, 0);
+        final IgniteObject po = new IgniteObjectImpl(initPortableContext(new PortableMarshaller()), arr, 0);
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
                                        @Override public Object call() throws Exception {
@@ -730,7 +730,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
                                            return null;
                                        }
-                                   }, PortableInvalidClassException.class, "Unknown type ID: 11111"
+                                   }, IgniteObjectInvalidClassException.class, "Unknown type ID: 11111"
         );
     }
 
@@ -741,25 +741,25 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-                                        new PortableTypeConfiguration(NoPublicConstructor.class.getName()),
-                                        new PortableTypeConfiguration(NoPublicDefaultConstructor.class.getName()),
-                                        new PortableTypeConfiguration(ProtectedConstructor.class.getName()))
+                                        new IgniteObjectConfiguration(NoPublicConstructor.class.getName()),
+                                        new IgniteObjectConfiguration(NoPublicDefaultConstructor.class.getName()),
+                                        new IgniteObjectConfiguration(ProtectedConstructor.class.getName()))
         );
 
         initPortableContext(marsh);
 
         NoPublicConstructor npc = new NoPublicConstructor();
-        PortableObject npc2 = marshal(npc, marsh);
+        IgniteObject npc2 = marshal(npc, marsh);
 
         assertEquals("test", npc2.<NoPublicConstructor>deserialize().val);
 
         NoPublicDefaultConstructor npdc = new NoPublicDefaultConstructor(239);
-        PortableObject npdc2 = marshal(npdc, marsh);
+        IgniteObject npdc2 = marshal(npdc, marsh);
 
         assertEquals(239, npdc2.<NoPublicDefaultConstructor>deserialize().val);
 
         ProtectedConstructor pc = new ProtectedConstructor();
-        PortableObject pc2 = marshal(pc, marsh);
+        IgniteObject pc2 = marshal(pc, marsh);
 
         assertEquals(ProtectedConstructor.class, pc2.<ProtectedConstructor>deserialize().getClass());
     }
@@ -770,8 +770,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testCustomSerializer() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration type =
-            new PortableTypeConfiguration(CustomSerializedObject1.class.getName());
+        IgniteObjectConfiguration type =
+            new IgniteObjectConfiguration(CustomSerializedObject1.class.getName());
 
         type.setSerializer(new CustomSerializer1());
 
@@ -779,7 +779,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
 
-        PortableObject po1 = marshal(obj1, marsh);
+        IgniteObject po1 = marshal(obj1, marsh);
 
         assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
     }
@@ -792,10 +792,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setSerializer(new CustomSerializer1());
 
-        PortableTypeConfiguration type1 =
-            new PortableTypeConfiguration(CustomSerializedObject1.class.getName());
-        PortableTypeConfiguration type2 =
-            new PortableTypeConfiguration(CustomSerializedObject2.class.getName());
+        IgniteObjectConfiguration type1 =
+            new IgniteObjectConfiguration(CustomSerializedObject1.class.getName());
+        IgniteObjectConfiguration type2 =
+            new IgniteObjectConfiguration(CustomSerializedObject2.class.getName());
 
         type2.setSerializer(new CustomSerializer2());
 
@@ -803,13 +803,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomSerializedObject1 obj1 = new CustomSerializedObject1(10);
 
-        PortableObject po1 = marshal(obj1, marsh);
+        IgniteObject po1 = marshal(obj1, marsh);
 
         assertEquals(20, po1.<CustomSerializedObject1>deserialize().val);
 
         CustomSerializedObject2 obj2 = new CustomSerializedObject2(10);
 
-        PortableObject po2 = marshal(obj2, marsh);
+        IgniteObject po2 = marshal(obj2, marsh);
 
         assertEquals(30, po2.<CustomSerializedObject2>deserialize().val);
     }
@@ -820,10 +820,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testCustomIdMapper() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration type =
-            new PortableTypeConfiguration(CustomMappedObject1.class.getName());
+        IgniteObjectConfiguration type =
+            new IgniteObjectConfiguration(CustomMappedObject1.class.getName());
 
-        type.setIdMapper(new PortableIdMapper() {
+        type.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 11111;
             }
@@ -846,7 +846,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str");
 
-        PortableObject po1 = marshal(obj1, marsh);
+        IgniteObject po1 = marshal(obj1, marsh);
 
         assertEquals(11111, po1.typeId());
         assertEquals(22222, intFromPortable(po1, 19));
@@ -862,7 +862,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testCustomIdMapperWithGlobal() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 11111;
             }
@@ -879,12 +879,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration type1 =
-            new PortableTypeConfiguration(CustomMappedObject1.class.getName());
-        PortableTypeConfiguration type2 =
-            new PortableTypeConfiguration(CustomMappedObject2.class.getName());
+        IgniteObjectConfiguration type1 =
+            new IgniteObjectConfiguration(CustomMappedObject1.class.getName());
+        IgniteObjectConfiguration type2 =
+            new IgniteObjectConfiguration(CustomMappedObject2.class.getName());
 
-        type2.setIdMapper(new PortableIdMapper() {
+        type2.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 44444;
             }
@@ -905,7 +905,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject1 obj1 = new CustomMappedObject1(10, "str1");
 
-        PortableObject po1 = marshal(obj1, marsh);
+        IgniteObject po1 = marshal(obj1, marsh);
 
         assertEquals(11111, po1.typeId());
         assertEquals(22222, intFromPortable(po1, 19));
@@ -916,7 +916,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CustomMappedObject2 obj2 = new CustomMappedObject2(20, "str2");
 
-        PortableObject po2 = marshal(obj2, marsh);
+        IgniteObject po2 = marshal(obj2, marsh);
 
         assertEquals(44444, po2.typeId());
         assertEquals(55555, intFromPortable(po2, 19));
@@ -933,10 +933,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(DynamicObject.class.getName())
+            new IgniteObjectConfiguration(DynamicObject.class.getName())
         ));
 
-        PortableObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
+        IgniteObject po1 = marshal(new DynamicObject(0, 10, 20, 30), marsh);
 
         assertEquals(new Integer(10), po1.field("val1"));
         assertEquals(null, po1.field("val2"));
@@ -948,7 +948,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals(0, do1.val2);
         assertEquals(0, do1.val3);
 
-        PortableObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
+        IgniteObject po2 = marshal(new DynamicObject(1, 10, 20, 30), marsh);
 
         assertEquals(new Integer(10), po2.field("val1"));
         assertEquals(new Integer(20), po2.field("val2"));
@@ -960,7 +960,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals(20, do2.val2);
         assertEquals(0, do2.val3);
 
-        PortableObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
+        IgniteObject po3 = marshal(new DynamicObject(2, 10, 20, 30), marsh);
 
         assertEquals(new Integer(10), po3.field("val1"));
         assertEquals(new Integer(20), po3.field("val2"));
@@ -980,14 +980,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(CycleLinkObject.class.getName())
+            new IgniteObjectConfiguration(CycleLinkObject.class.getName())
         ));
 
         CycleLinkObject obj = new CycleLinkObject();
 
         obj.self = obj;
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         CycleLinkObject obj0 = po.deserialize();
 
@@ -1001,8 +1001,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(DetachedTestObject.class.getName()),
-            new PortableTypeConfiguration(DetachedInnerTestObject.class.getName())
+            new IgniteObjectConfiguration(DetachedTestObject.class.getName()),
+            new IgniteObjectConfiguration(DetachedInnerTestObject.class.getName())
         ));
 
         UUID id = UUID.randomUUID();
@@ -1015,7 +1015,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assert obj.inner1 == obj.inner4;
 
-        PortableObjectImpl innerPo = (PortableObjectImpl)obj.inner2;
+        IgniteObjectImpl innerPo = (IgniteObjectImpl)obj.inner2;
 
         assert innerPo.detached();
 
@@ -1023,7 +1023,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assertEquals(id, inner.id);
 
-        PortableObjectImpl detachedPo = (PortableObjectImpl)innerPo.detach();
+        IgniteObjectImpl detachedPo = (IgniteObjectImpl)innerPo.detach();
 
         assert detachedPo.detached();
 
@@ -1031,7 +1031,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         assertEquals(id, inner.id);
 
-        innerPo = (PortableObjectImpl)obj.inner3;
+        innerPo = (IgniteObjectImpl)obj.inner3;
 
         assert innerPo.detached();
 
@@ -1040,7 +1040,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals(id, inner.id);
         assertNotNull(inner.inner);
 
-        detachedPo = (PortableObjectImpl)innerPo.detach();
+        detachedPo = (IgniteObjectImpl)innerPo.detach();
 
         assert detachedPo.detached();
 
@@ -1057,9 +1057,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(CollectionFieldsObject.class.getName()),
-            new PortableTypeConfiguration(Key.class.getName()),
-            new PortableTypeConfiguration(Value.class.getName())
+            new IgniteObjectConfiguration(CollectionFieldsObject.class.getName()),
+            new IgniteObjectConfiguration(Key.class.getName()),
+            new IgniteObjectConfiguration(Value.class.getName())
         ));
 
         Object[] arr = new Object[] {new Value(1), new Value(2), new Value(3)};
@@ -1068,7 +1068,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         CollectionFieldsObject obj = new CollectionFieldsObject(arr, col, map);
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         Object[] arr0 = po.field("arr");
 
@@ -1077,18 +1077,18 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         int i = 1;
 
         for (Object valPo : arr0)
-            assertEquals(i++, ((PortableObject)valPo).<Value>deserialize().val);
+            assertEquals(i++, ((IgniteObject)valPo).<Value>deserialize().val);
 
-        Collection<PortableObject> col0 = po.field("col");
+        Collection<IgniteObject> col0 = po.field("col");
 
         i = 4;
 
-        for (PortableObject valPo : col0)
+        for (IgniteObject valPo : col0)
             assertEquals(i++, valPo.<Value>deserialize().val);
 
-        Map<PortableObject, PortableObject> map0 = po.field("map");
+        Map<IgniteObject, IgniteObject> map0 = po.field("map");
 
-        for (Map.Entry<PortableObject, PortableObject> e : map0.entrySet())
+        for (Map.Entry<IgniteObject, IgniteObject> e : map0.entrySet())
             assertEquals(e.getKey().<Key>deserialize().key, e.getValue().<Value>deserialize().val);
     }
 
@@ -1098,10 +1098,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testDefaultMapping() throws Exception {
         PortableMarshaller marsh1 = new PortableMarshaller();
 
-        PortableTypeConfiguration customMappingType =
-            new PortableTypeConfiguration(TestPortableObject.class.getName());
+        IgniteObjectConfiguration customMappingType =
+            new IgniteObjectConfiguration(TestIgniteObjectObject.class.getName());
 
-        customMappingType.setIdMapper(new PortableIdMapper() {
+        customMappingType.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 String typeName;
 
@@ -1125,19 +1125,19 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         });
 
         marsh1.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
+            new IgniteObjectConfiguration(SimpleObject.class.getName()),
             customMappingType
         ));
 
-        TestPortableObject obj = portableObject();
+        TestIgniteObjectObject obj = portableObject();
 
-        PortableObjectImpl po = marshal(obj, marsh1);
+        IgniteObjectImpl po = marshal(obj, marsh1);
 
         PortableMarshaller marsh2 = new PortableMarshaller();
 
         marsh2.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName()),
-            new PortableTypeConfiguration(TestPortableObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName()),
+            new IgniteObjectConfiguration(TestIgniteObjectObject.class.getName())
         ));
 
         PortableContext ctx = initPortableContext(marsh2);
@@ -1153,9 +1153,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testTypeNames() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration(Value.class.getName());
+        IgniteObjectConfiguration customType1 = new IgniteObjectConfiguration(Value.class.getName());
 
-        customType1.setIdMapper(new PortableIdMapper() {
+        customType1.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 300;
             }
@@ -1165,9 +1165,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("org.gridgain.NonExistentClass1");
+        IgniteObjectConfiguration customType2 = new IgniteObjectConfiguration("org.gridgain.NonExistentClass1");
 
-        customType2.setIdMapper(new PortableIdMapper() {
+        customType2.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 400;
             }
@@ -1177,9 +1177,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration customType3 = new PortableTypeConfiguration("NonExistentClass2");
+        IgniteObjectConfiguration customType3 = new IgniteObjectConfiguration("NonExistentClass2");
 
-        customType3.setIdMapper(new PortableIdMapper() {
+        customType3.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 500;
             }
@@ -1189,9 +1189,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration customType4 = new PortableTypeConfiguration("NonExistentClass5");
+        IgniteObjectConfiguration customType4 = new IgniteObjectConfiguration("NonExistentClass5");
 
-        customType4.setIdMapper(new PortableIdMapper() {
+        customType4.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 0;
             }
@@ -1202,9 +1202,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         });
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(Key.class.getName()),
-            new PortableTypeConfiguration("org.gridgain.NonExistentClass3"),
-            new PortableTypeConfiguration("NonExistentClass4"),
+            new IgniteObjectConfiguration(Key.class.getName()),
+            new IgniteObjectConfiguration("org.gridgain.NonExistentClass3"),
+            new IgniteObjectConfiguration("NonExistentClass4"),
             customType1,
             customType2,
             customType3,
@@ -1229,9 +1229,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testFieldIdMapping() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration(Value.class.getName());
+        IgniteObjectConfiguration customType1 = new IgniteObjectConfiguration(Value.class.getName());
 
-        customType1.setIdMapper(new PortableIdMapper() {
+        customType1.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 300;
             }
@@ -1250,9 +1250,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("NonExistentClass1");
+        IgniteObjectConfiguration customType2 = new IgniteObjectConfiguration("NonExistentClass1");
 
-        customType2.setIdMapper(new PortableIdMapper() {
+        customType2.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 400;
             }
@@ -1271,8 +1271,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(Key.class.getName()),
-                                                  new PortableTypeConfiguration("NonExistentClass2"),
+        marsh.setTypeConfigurations(Arrays.asList(new IgniteObjectConfiguration(Key.class.getName()),
+                                                  new IgniteObjectConfiguration("NonExistentClass2"),
                                                   customType1,
                                                   customType2));
 
@@ -1295,9 +1295,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testDuplicateTypeId() throws Exception {
         final PortableMarshaller marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration customType1 = new PortableTypeConfiguration("org.gridgain.Class1");
+        IgniteObjectConfiguration customType1 = new IgniteObjectConfiguration("org.gridgain.Class1");
 
-        customType1.setIdMapper(new PortableIdMapper() {
+        customType1.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 100;
             }
@@ -1307,9 +1307,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             }
         });
 
-        PortableTypeConfiguration customType2 = new PortableTypeConfiguration("org.gridgain.Class2");
+        IgniteObjectConfiguration customType2 = new IgniteObjectConfiguration("org.gridgain.Class2");
 
-        customType2.setIdMapper(new PortableIdMapper() {
+        customType2.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 100;
             }
@@ -1341,14 +1341,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        final PortableObject po = marshal(obj, marsh);
+        final IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, null);
+        IgniteObject copy = copy(po, null);
 
         assertEquals(obj, copy.deserialize());
 
@@ -1464,14 +1464,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("str", "str3"));
 
         assertEquals("str3", copy.<String>field("str"));
 
@@ -1487,16 +1487,16 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         UUID uuid = UUID.randomUUID();
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("uuid", uuid));
 
         assertEquals(uuid, copy.<UUID>field("uuid"));
 
@@ -1512,14 +1512,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("bArr", new byte[]{1, 2, 3}));
 
         assertArrayEquals(new byte[] {1, 2, 3}, copy.<byte[]>field("bArr"));
 
@@ -1533,8 +1533,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param fields Fields.
      * @return Copy.
      */
-    private PortableObject copy(PortableObject po, Map<String, Object> fields) {
-        PortableBuilder builder = PortableBuilderImpl.wrap(po);
+    private IgniteObject copy(IgniteObject po, Map<String, Object> fields) {
+        IgniteObjectBuilder builder = IgniteObjectBuilderImpl.wrap(po);
 
         if (fields != null) {
             for (Map.Entry<String, Object> e : fields.entrySet())
@@ -1551,14 +1551,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("sArr", new short[]{1, 2, 3}));
 
         assertArrayEquals(new short[] {1, 2, 3}, copy.<short[]>field("sArr"));
 
@@ -1574,14 +1574,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("iArr", new int[]{1, 2, 3}));
 
         assertArrayEquals(new int[] {1, 2, 3}, copy.<int[]>field("iArr"));
 
@@ -1597,14 +1597,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("lArr", new long[]{1, 2, 3}));
 
         assertArrayEquals(new long[] {1, 2, 3}, copy.<long[]>field("lArr"));
 
@@ -1620,14 +1620,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("fArr", new float[]{1, 2, 3}));
 
         assertArrayEquals(new float[] {1, 2, 3}, copy.<float[]>field("fArr"), 0);
 
@@ -1643,14 +1643,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("dArr", new double[]{1, 2, 3}));
 
         assertArrayEquals(new double[] {1, 2, 3}, copy.<double[]>field("dArr"), 0);
 
@@ -1666,14 +1666,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("cArr", new char[]{1, 2, 3}));
 
         assertArrayEquals(new char[]{1, 2, 3}, copy.<char[]>field("cArr"));
 
@@ -1689,14 +1689,14 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("strArr", new String[]{"str1", "str2"}));
 
         assertArrayEquals(new String[]{"str1", "str2"}, copy.<String[]>field("strArr"));
 
@@ -1712,12 +1712,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         SimpleObject newObj = new SimpleObject();
 
@@ -1725,9 +1725,9 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         newObj.fArr = new float[] {5, 8, 0};
         newObj.str = "newStr";
 
-        PortableObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
+        IgniteObject copy = copy(po, F.<String, Object>asMap("inner", newObj));
 
-        assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
+        assertEquals(newObj, copy.<IgniteObject>field("inner").deserialize());
 
         SimpleObject obj0 = copy.deserialize();
 
@@ -1741,12 +1741,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())
+            new IgniteObjectConfiguration(SimpleObject.class.getName())
         ));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         Map<String, Object> map = new HashMap<>(3, 1.0f);
 
@@ -1760,10 +1760,10 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         map.put("inner", newObj);
         map.put("bArr", new byte[]{6, 7, 9});
 
-        PortableObject copy = copy(po, map);
+        IgniteObject copy = copy(po, map);
 
         assertEquals("str555", copy.<String>field("str"));
-        assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
+        assertEquals(newObj, copy.<IgniteObject>field("inner").deserialize());
         assertArrayEquals(new byte[]{6, 7, 9}, copy.<byte[]>field("bArr"));
 
         SimpleObject obj0 = copy.deserialize();
@@ -1779,11 +1779,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testPortableCopyMixed() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
+        marsh.setTypeConfigurations(Arrays.asList(new IgniteObjectConfiguration(SimpleObject.class.getName())));
 
         SimpleObject obj = simpleObject();
 
-        PortableObject po = marshal(obj, marsh);
+        IgniteObject po = marshal(obj, marsh);
 
         Map<String, Object> map = new HashMap<>(3, 1.0f);
 
@@ -1800,11 +1800,11 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         map.put("bArr", new byte[]{6, 7, 9});
         map.put("b", (byte)111);
 
-        PortableObject copy = copy(po, map);
+        IgniteObject copy = copy(po, map);
 
         assertEquals(1234, copy.<Integer>field("i").intValue());
         assertEquals("str555", copy.<String>field("str"));
-        assertEquals(newObj, copy.<PortableObject>field("inner").deserialize());
+        assertEquals(newObj, copy.<IgniteObject>field("inner").deserialize());
         assertEquals((short)2323, copy.<Short>field("s").shortValue());
         assertArrayEquals(new byte[] {6, 7, 9}, copy.<byte[]>field("bArr"));
         assertEquals((byte)111, copy.<Byte>field("b").byteValue());
@@ -1828,7 +1828,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         marsh.setClassNames(Arrays.asList(SimpleObject.class.getName()));
         marsh.setKeepDeserialized(true);
 
-        PortableObject po = marshal(simpleObject(), marsh);
+        IgniteObject po = marshal(simpleObject(), marsh);
 
         assert po.deserialize() == po.deserialize();
 
@@ -1845,7 +1845,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setKeepDeserialized(true);
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())));
+            new IgniteObjectConfiguration(SimpleObject.class.getName())));
 
         po = marshal(simpleObject(), marsh);
 
@@ -1855,7 +1855,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setKeepDeserialized(false);
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(SimpleObject.class.getName())));
+            new IgniteObjectConfiguration(SimpleObject.class.getName())));
 
         po = marshal(simpleObject(), marsh);
 
@@ -1865,7 +1865,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setKeepDeserialized(true);
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(SimpleObject.class.getName());
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(SimpleObject.class.getName());
 
         typeCfg.setKeepDeserialized(false);
 
@@ -1879,7 +1879,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setKeepDeserialized(false);
 
-        typeCfg = new PortableTypeConfiguration(SimpleObject.class.getName());
+        typeCfg = new IgniteObjectConfiguration(SimpleObject.class.getName());
 
         typeCfg.setKeepDeserialized(true);
 
@@ -1896,13 +1896,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     public void testOffheapPortable() throws Exception {
         PortableMarshaller marsh = new PortableMarshaller();
 
-        marsh.setTypeConfigurations(Arrays.asList(new PortableTypeConfiguration(SimpleObject.class.getName())));
+        marsh.setTypeConfigurations(Arrays.asList(new IgniteObjectConfiguration(SimpleObject.class.getName())));
 
         PortableContext ctx = initPortableContext(marsh);
 
         SimpleObject simpleObj = simpleObject();
 
-        PortableObjectImpl obj = marshal(simpleObj, marsh);
+        IgniteObjectImpl obj = marshal(simpleObj, marsh);
 
         long ptr = 0;
 
@@ -1913,7 +1913,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         try {
             ptr = copyOffheap(obj);
 
-            PortableObjectOffheapImpl offheapObj = new PortableObjectOffheapImpl(ctx,
+            IgniteObjectOffheapImpl offheapObj = new IgniteObjectOffheapImpl(ctx,
                 ptr,
                 0,
                 obj.array().length);
@@ -1926,7 +1926,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
             ptr1 = copyOffheap(obj);
 
-            PortableObjectOffheapImpl offheapObj1 = new PortableObjectOffheapImpl(ctx,
+            IgniteObjectOffheapImpl offheapObj1 = new IgniteObjectOffheapImpl(ctx,
                 ptr1,
                 0,
                 obj.array().length);
@@ -1939,20 +1939,20 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
             checkSimpleObjectData(simpleObj, offheapObj);
 
-            PortableObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
+            IgniteObjectOffheapImpl innerOffheapObj = offheapObj.field("inner");
 
             assertNotNull(innerOffheapObj);
 
             checkSimpleObjectData(simpleObj.inner, innerOffheapObj);
 
-            obj = (PortableObjectImpl)offheapObj.heapCopy();
+            obj = (IgniteObjectImpl)offheapObj.heapCopy();
 
             assertEquals(obj.typeId(), offheapObj.typeId());
             assertEquals(obj.hashCode(), offheapObj.hashCode());
 
             checkSimpleObjectData(simpleObj, obj);
 
-            PortableObjectImpl innerObj = obj.field("inner");
+            IgniteObjectImpl innerObj = obj.field("inner");
 
             assertNotNull(innerObj);
 
@@ -1967,7 +1967,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
             ptr2 = copyOffheap(obj);
 
-            PortableObjectOffheapImpl offheapObj2 = new PortableObjectOffheapImpl(ctx,
+            IgniteObjectOffheapImpl offheapObj2 = new IgniteObjectOffheapImpl(ctx,
                 ptr2,
                 0,
                 obj.array().length);
@@ -1995,7 +1995,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         marsh.setClassNames(
             Arrays.asList(MySingleton.class.getName(), SingletonMarker.class.getName()));
 
-        PortableObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
+        IgniteObjectImpl portableObj = marshal(MySingleton.INSTANCE, marsh);
 
         assertTrue(portableObj.array().length <= 1024); // Check that big string was not serialized.
 
@@ -2012,7 +2012,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setClassNames(Collections.singletonList(MyTestClass.class.getName()));
 
-        PortableObjectImpl portableObj = marshal(new MyTestClass(), marsh);
+        IgniteObjectImpl portableObj = marshal(new MyTestClass(), marsh);
 
         MyTestClass obj = portableObj.deserialize();
 
@@ -2027,7 +2027,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         marsh.setClassNames(Arrays.asList(ChildPortable.class.getName()));
 
-        PortableObjectImpl portableObj = marshal(new ChildPortable(), marsh);
+        IgniteObjectImpl portableObj = marshal(new ChildPortable(), marsh);
 
         ChildPortable singleton = portableObj.deserialize();
 
@@ -2053,7 +2053,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         obj1.val = BigDecimal.ZERO;
         obj1.valArr = new BigDecimal[] { BigDecimal.ONE, BigDecimal.TEN };
 
-        PortableObjectImpl portObj = marshal(obj1, marsh);
+        IgniteObjectImpl portObj = marshal(obj1, marsh);
 
         assertEquals(obj1.val, portObj.field("val"));
         assertArrayEquals(obj1.valArr, portObj.<BigDecimal[]>field("valArr"));
@@ -2100,7 +2100,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         // Checking the writer directly.
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
-        try (PortableWriterExImpl writer = new PortableWriterExImpl(initPortableContext(new PortableMarshaller()), 0)) {
+        try (IgniteObjectWriterExImpl writer = new IgniteObjectWriterExImpl(initPortableContext(new PortableMarshaller()), 0)) {
             assertEquals(true, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
             writer.writeString("Thread local test");
@@ -2122,12 +2122,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
 
         // Checking the builder.
-        PortableBuilder builder = new PortableBuilderImpl(initPortableContext(new PortableMarshaller()),
+        IgniteObjectBuilder builder = new IgniteObjectBuilderImpl(initPortableContext(new PortableMarshaller()),
             "org.gridgain.foo.bar.TestClass");
 
         builder.setField("a", "1");
 
-        PortableObject portableObj = builder.build();
+        IgniteObject portableObj = builder.build();
 
         assertEquals(false, THREAD_LOCAL_ALLOC.isThreadLocalArrayAcquired());
     }
@@ -2147,7 +2147,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
         try {
             marsh.marshal(job2);
-        } catch (PortableException e) {
+        } catch (IgniteObjectException e) {
             assertEquals(true, e.getMessage().contains("Failed to register class"));
             return;
         }
@@ -2339,7 +2339,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param obj Object.
      * @return Offheap address.
      */
-    private long copyOffheap(PortableObjectImpl obj) {
+    private long copyOffheap(IgniteObjectImpl obj) {
         byte[] arr = obj.array();
 
         long ptr = UNSAFE.allocateMemory(arr.length);
@@ -2367,7 +2367,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param off Offset.
      * @return Value.
      */
-    private int intFromPortable(PortableObject po, int off) {
+    private int intFromPortable(IgniteObject po, int off) {
         byte[] arr = U.field(po, "arr");
 
         return Integer.reverseBytes(U.bytesToInt(arr, off));
@@ -2399,12 +2399,12 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
      * @param marsh Marshaller.
      * @return Portable object.
      */
-    private <T> PortableObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
+    private <T> IgniteObjectImpl marshal(T obj, PortableMarshaller marsh) throws IgniteCheckedException {
         initPortableContext(marsh);
 
         byte[] bytes = marsh.marshal(obj);
 
-        return new PortableObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
+        return new IgniteObjectImpl(U.<GridPortableMarshaller>field(marsh, "impl").context(),
             bytes, 0);
     }
 
@@ -2536,7 +2536,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      * @return Portable object.
      */
-    private TestPortableObject portableObject() {
+    private TestIgniteObjectObject portableObject() {
         SimpleObject innerSimple = new SimpleObject();
 
         innerSimple.b = 1;
@@ -2576,7 +2576,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         innerSimple.map.put(2, "str2");
         innerSimple.map.put(3, "str3");
 
-        TestPortableObject innerPortable = new TestPortableObject();
+        TestIgniteObjectObject innerPortable = new TestIgniteObjectObject();
 
         innerPortable.b = 2;
         innerPortable.s = 2;
@@ -2651,7 +2651,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         innerPortable.mapRaw.put(8, "str8");
         innerPortable.mapRaw.put(9, "str9");
 
-        TestPortableObject outer = new TestPortableObject();
+        TestIgniteObjectObject outer = new TestIgniteObjectObject();
 
         outer.b = 4;
         outer.s = 4;
@@ -2855,7 +2855,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     }
 
     /** */
-    private static class TestPortableObject implements PortableMarshalAware {
+    private static class TestIgniteObjectObject implements IgniteObjectMarshalAware {
         /** */
         private byte b;
 
@@ -3031,13 +3031,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         private SimpleObject simpleRaw;
 
         /** */
-        private TestPortableObject portable;
+        private TestIgniteObjectObject portable;
 
         /** */
-        private TestPortableObject portableRaw;
+        private TestIgniteObjectObject portableRaw;
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeByte("_b", b);
             writer.writeShort("_s", s);
             writer.writeInt("_i", i);
@@ -3069,7 +3069,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             writer.writeObject("_simple", simple);
             writer.writeObject("_portable", portable);
 
-            PortableRawWriter raw = writer.rawWriter();
+            IgniteObjectRawWriter raw = writer.rawWriter();
 
             raw.writeByte(bRaw);
             raw.writeShort(sRaw);
@@ -3104,7 +3104,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             b = reader.readByte("_b");
             s = reader.readShort("_s");
             i = reader.readInt("_i");
@@ -3136,7 +3136,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             simple = reader.readObject("_simple");
             portable = reader.readObject("_portable");
 
-            PortableRawReader raw = reader.rawReader();
+            IgniteObjectRawReader raw = reader.rawReader();
 
             bRaw = raw.readByte();
             sRaw = raw.readShort();
@@ -3179,20 +3179,20 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
             if (other == null || getClass() != other.getClass())
                 return false;
 
-            TestPortableObject obj = (TestPortableObject)other;
+            TestIgniteObjectObject obj = (TestIgniteObjectObject)other;
 
             return GridTestUtils.deepEquals(this, obj);
         }
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            return S.toString(TestPortableObject.class, this);
+            return S.toString(TestIgniteObjectObject.class, this);
         }
     }
 
     /**
      */
-    private static class CustomSerializedObject1 implements PortableMarshalAware {
+    private static class CustomSerializedObject1 implements IgniteObjectMarshalAware {
         /** */
         private int val;
 
@@ -3210,19 +3210,19 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             // No-op.
         }
     }
 
     /**
      */
-    private static class CustomSerializedObject2 implements PortableMarshalAware {
+    private static class CustomSerializedObject2 implements IgniteObjectMarshalAware {
         /** */
         private int val;
 
@@ -3240,28 +3240,28 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             // No-op.
         }
     }
 
     /**
      */
-    private static class CustomSerializer1 implements PortableSerializer {
+    private static class CustomSerializer1 implements IgniteObjectSerializer {
         /** {@inheritDoc} */
-        @Override public void writePortable(Object obj, PortableWriter writer) throws PortableException {
+        @Override public void writePortable(Object obj, IgniteObjectWriter writer) throws IgniteObjectException {
             CustomSerializedObject1 o = (CustomSerializedObject1)obj;
 
             writer.writeInt("val", o.val * 2);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(Object obj, PortableReader reader) throws PortableException {
+        @Override public void readPortable(Object obj, IgniteObjectReader reader) throws IgniteObjectException {
             CustomSerializedObject1 o = (CustomSerializedObject1)obj;
 
             o.val = reader.readInt("val");
@@ -3270,16 +3270,16 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class CustomSerializer2 implements PortableSerializer {
+    private static class CustomSerializer2 implements IgniteObjectSerializer {
         /** {@inheritDoc} */
-        @Override public void writePortable(Object obj, PortableWriter writer) throws PortableException {
+        @Override public void writePortable(Object obj, IgniteObjectWriter writer) throws IgniteObjectException {
             CustomSerializedObject2 o = (CustomSerializedObject2)obj;
 
             writer.writeInt("val", o.val * 3);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(Object obj, PortableReader reader) throws PortableException {
+        @Override public void readPortable(Object obj, IgniteObjectReader reader) throws IgniteObjectException {
             CustomSerializedObject2 o = (CustomSerializedObject2)obj;
 
             o.val = reader.readInt("val");
@@ -3338,7 +3338,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class DynamicObject implements PortableMarshalAware {
+    private static class DynamicObject implements IgniteObjectMarshalAware {
         /** */
         private int idx;
 
@@ -3370,7 +3370,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("val1", val1);
 
             if (idx > 0)
@@ -3383,7 +3383,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val1 = reader.readInt("val1");
             val2 = reader.readInt("val2");
             val3 = reader.readInt("val3");
@@ -3399,7 +3399,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class DetachedTestObject implements PortableMarshalAware {
+    private static class DetachedTestObject implements IgniteObjectMarshalAware {
         /** */
         private DetachedInnerTestObject inner1;
 
@@ -3429,8 +3429,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
-            PortableRawWriterEx raw = (PortableRawWriterEx)writer.rawWriter();
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+            IgniteObjectRawWriterEx raw = (IgniteObjectRawWriterEx)writer.rawWriter();
 
             raw.writeObject(inner1);
             raw.writeObjectDetached(inner2);
@@ -3439,8 +3439,8 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
-            PortableRawReaderEx raw = (PortableRawReaderEx)reader.rawReader();
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+            IgniteObjectRawReaderEx raw = (IgniteObjectRawReaderEx)reader.rawReader();
 
             inner1 = (DetachedInnerTestObject)raw.readObject();
             inner2 = raw.readObjectDetached();
@@ -3620,7 +3620,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    private static class MyTestClass implements PortableMarshalAware {
+    private static class MyTestClass implements IgniteObjectMarshalAware {
         /** */
         private boolean readyToSerialize;
 
@@ -3646,13 +3646,13 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             if (!readyToSerialize)
                 fail();
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             s = "readPortable";
         }
     }
@@ -3737,7 +3737,7 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    private static class DecimalMarshalAware extends DecimalReflective implements PortableMarshalAware {
+    private static class DecimalMarshalAware extends DecimalReflective implements IgniteObjectMarshalAware {
         /** */
         public BigDecimal rawVal;
 
@@ -3745,22 +3745,22 @@ public class GridPortableMarshallerSelfTest extends GridCommonAbstractTest {
         public BigDecimal[] rawValArr;
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeDecimal("val", val);
             writer.writeDecimalArray("valArr", valArr);
 
-            PortableRawWriter rawWriter = writer.rawWriter();
+            IgniteObjectRawWriter rawWriter = writer.rawWriter();
 
             rawWriter.writeDecimal(rawVal);
             rawWriter.writeDecimalArray(rawValArr);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             val = reader.readDecimal("val");
             valArr = reader.readDecimalArray("valArr");
 
-            PortableRawReader rawReader = reader.rawReader();
+            IgniteObjectRawReader rawReader = reader.rawReader();
 
             rawVal = rawReader.readDecimal();
             rawValArr = rawReader.readDecimalArray();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
index 05df23b..8800d81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataDisabledSelfTest.java
@@ -17,15 +17,15 @@
 package org.apache.ignite.internal.portable;
 
 import java.util.Arrays;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -47,7 +47,7 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
     /**
      * @return Portables.
      */
-    private IgnitePortables portables() {
+    private IgniteObjects portables() {
         return grid().portables();
     }
 
@@ -74,7 +74,7 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
             assertEquals(0, portables().metadata(TestObject1.class).fields().size());
             assertEquals(0, portables().metadata(TestObject2.class).fields().size());
 
-            PortableBuilder bldr = portables().builder("FakeType");
+            IgniteObjectBuilder bldr = portables().builder("FakeType");
 
             bldr.setField("field1", 0).setField("field2", "value").build();
 
@@ -92,12 +92,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
     public void testDisableGlobalSimpleClass() throws Exception {
         marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject2.class.getName());
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(TestObject2.class.getName());
 
         typeCfg.setMetaDataEnabled(true);
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject1.class.getName()),
+            new IgniteObjectConfiguration(TestObject1.class.getName()),
             typeCfg
         ));
 
@@ -123,12 +123,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
     public void testDisableGlobalMarshalAwareClass() throws Exception {
         marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject1.class.getName());
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(TestObject1.class.getName());
 
         typeCfg.setMetaDataEnabled(true);
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject2.class.getName()),
+            new IgniteObjectConfiguration(TestObject2.class.getName()),
             typeCfg
         ));
 
@@ -154,12 +154,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
     public void testDisableSimpleClass() throws Exception {
         marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject1.class.getName());
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(TestObject1.class.getName());
 
         typeCfg.setMetaDataEnabled(false);
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject2.class.getName()),
+            new IgniteObjectConfiguration(TestObject2.class.getName()),
             typeCfg
         ));
 
@@ -183,12 +183,12 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
     public void testDisableMarshalAwareClass() throws Exception {
         marsh = new PortableMarshaller();
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(TestObject2.class.getName());
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(TestObject2.class.getName());
 
         typeCfg.setMetaDataEnabled(false);
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration(TestObject1.class.getName()),
+            new IgniteObjectConfiguration(TestObject1.class.getName()),
             typeCfg
         ));
 
@@ -216,14 +216,14 @@ public class GridPortableMetaDataDisabledSelfTest extends GridCommonAbstractTest
 
     /**
      */
-    private static class TestObject2 implements PortableMarshalAware {
+    private static class TestObject2 implements IgniteObjectMarshalAware {
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeInt("field", 1);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
index fa3c9a7..ac9645b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMetaDataSelfTest.java
@@ -22,17 +22,17 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Date;
 import java.util.HashMap;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.IgniteObjects;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -74,7 +74,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     /**
      * @return Portables API.
      */
-    protected IgnitePortables portables() {
+    protected IgniteObjects portables() {
         return grid().portables();
     }
 
@@ -84,11 +84,11 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     public void testGetAll() throws Exception {
         portables().toPortable(new TestObject2());
 
-        Collection<PortableMetadata> metas = portables().metadata();
+        Collection<IgniteObjectMetadata> metas = portables().metadata();
 
         assertEquals(2, metas.size());
 
-        for (PortableMetadata meta : metas) {
+        for (IgniteObjectMetadata meta : metas) {
             Collection<String> fields;
 
             switch (meta.typeName()) {
@@ -159,7 +159,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testReflection() throws Exception {
-        PortableMetadata meta = portables().metadata(TestObject1.class);
+        IgniteObjectMetadata meta = portables().metadata(TestObject1.class);
 
         assertNotNull(meta);
 
@@ -192,7 +192,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
     public void testPortableMarshalAware() throws Exception {
         portables().toPortable(new TestObject2());
 
-        PortableMetadata meta = portables().metadata(TestObject2.class);
+        IgniteObjectMetadata meta = portables().metadata(TestObject2.class);
 
         assertNotNull(meta);
 
@@ -229,7 +229,7 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
         portables().toPortable(new TestObject2());
 
-        PortableMetadata meta = portables().metadata(TestObject2.class);
+        IgniteObjectMetadata meta = portables().metadata(TestObject2.class);
 
         assertNotNull(meta);
 
@@ -274,11 +274,11 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
         obj.decVal = BigDecimal.ZERO;
         obj.decArrVal = new BigDecimal[] { BigDecimal.ONE };
 
-        PortableObject po = portables().toPortable(obj);
+        IgniteObject po = portables().toPortable(obj);
 
         info(po.toString());
 
-        PortableMetadata meta = po.metaData();
+        IgniteObjectMetadata meta = po.metaData();
 
         assertNotNull(meta);
 
@@ -333,9 +333,9 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
 
     /**
      */
-    private static class TestObject2 implements PortableMarshalAware {
+    private static class TestObject2 implements IgniteObjectMarshalAware {
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeBoolean("boolVal", false);
             writer.writeDate("dateVal", new Date());
             writer.writeUuidArray("uuidArrVal", null);
@@ -349,14 +349,14 @@ public class GridPortableMetaDataSelfTest extends GridCommonAbstractTest {
                 writer.writeCollection("colVal", null);
             }
 
-            PortableRawWriter raw = writer.rawWriter();
+            IgniteObjectRawWriter raw = writer.rawWriter();
 
             raw.writeChar((char)0);
             raw.writeCollection(null);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
index 349f152..b8b8189 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableWildcardsSelfTest.java
@@ -22,9 +22,9 @@ import java.util.Map;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -33,11 +33,11 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
     /** */
     private static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
+        @Override public void addMeta(int typeId, IgniteObjectMetadata meta) {
             // No-op.
         }
 
-        @Override public PortableMetadata metadata(int typeId) {
+        @Override public IgniteObjectMetadata metadata(int typeId) {
             return null;
         }
     };
@@ -91,7 +91,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -116,7 +116,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -134,8 +134,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = portableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.internal.portable.test.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
@@ -157,7 +157,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -176,13 +176,13 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         });
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.internal.portable.test.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -199,7 +199,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -218,13 +218,13 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         });
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.internal.portable.test.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.internal.portable.test.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -245,10 +245,10 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             "org.apache.ignite.internal.portable.test.*"
         ));
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration();
 
         typeCfg.setClassName("org.apache.ignite.internal.portable.test.GridPortableTestClass2");
-        typeCfg.setIdMapper(new PortableIdMapper() {
+        typeCfg.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 100;
             }
@@ -270,7 +270,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         assertTrue(typeIds.containsKey("innerclass".hashCode()));
         assertFalse(typeIds.containsKey("gridportabletestclass2".hashCode()));
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(100, typeMappers.get("GridPortableTestClass2").typeId("GridPortableTestClass2"));
     }
@@ -306,7 +306,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -329,7 +329,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -346,8 +346,8 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         PortableMarshaller marsh = portableMarshaller();
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.portable.testjar.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
@@ -368,7 +368,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -385,13 +385,13 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         });
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.portable.testjar.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -407,7 +407,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         PortableMarshaller marsh = portableMarshaller();
 
-        marsh.setIdMapper(new PortableIdMapper() {
+        marsh.setIdMapper(new IgniteObjectIdMapper() {
             @SuppressWarnings("IfMayBeConditional")
             @Override public int typeId(String clsName) {
                 if (clsName.endsWith("1"))
@@ -424,13 +424,13 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
         });
 
         marsh.setTypeConfigurations(Arrays.asList(
-            new PortableTypeConfiguration("org.apache.ignite.portable.testjar.*"),
-            new PortableTypeConfiguration("unknown.*")
+            new IgniteObjectConfiguration("org.apache.ignite.portable.testjar.*"),
+            new IgniteObjectConfiguration("unknown.*")
         ));
 
         ctx.configure(marsh);
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 
@@ -450,10 +450,10 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
             "org.apache.ignite.portable.testjar.*"
         ));
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration(
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration(
             "org.apache.ignite.portable.testjar.GridPortableTestClass2");
 
-        typeCfg.setIdMapper(new PortableIdMapper() {
+        typeCfg.setIdMapper(new IgniteObjectIdMapper() {
             @Override public int typeId(String clsName) {
                 return 100;
             }
@@ -473,7 +473,7 @@ public class GridPortableWildcardsSelfTest extends GridCommonAbstractTest {
 
         assertTrue(typeIds.containsKey("gridportabletestclass1".hashCode()));
 
-        Map<String, PortableIdMapper> typeMappers = U.field(ctx, "typeMappers");
+        Map<String, IgniteObjectIdMapper> typeMappers = U.field(ctx, "typeMappers");
 
         assertEquals(3, typeMappers.size());
 


[10/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java
new file mode 100644
index 0000000..014b487
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgniteObjectsImpl.java
@@ -0,0 +1,177 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.util.Collection;
+import org.apache.ignite.IgniteObjects;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * {@link org.apache.ignite.IgniteObjects} implementation.
+ */
+public class IgniteObjectsImpl implements IgniteObjects {
+    /** */
+    private GridKernalContext ctx;
+
+    /** */
+    private CacheObjectPortableProcessor proc;
+
+    /**
+     * @param ctx Context.
+     */
+    public IgniteObjectsImpl(GridKernalContext ctx, CacheObjectPortableProcessor proc) {
+        this.ctx = ctx;
+
+        this.proc = proc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int typeId(String typeName) {
+        guard();
+
+        try {
+            return proc.typeId(typeName);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T toPortable(@Nullable Object obj) throws IgniteObjectException {
+        guard();
+
+        try {
+            return (T)proc.marshalToPortable(obj);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObjectBuilder builder(int typeId) {
+        guard();
+
+        try {
+            return proc.builder(typeId);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObjectBuilder builder(String typeName) {
+        guard();
+
+        try {
+            return proc.builder(typeName);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObjectBuilder builder(IgniteObject portableObj) {
+        guard();
+
+        try {
+            return proc.builder(portableObj);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteObjectMetadata metadata(Class<?> cls) throws IgniteObjectException {
+        guard();
+
+        try {
+            return proc.metadata(proc.typeId(cls.getName()));
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteObjectMetadata metadata(String typeName) throws IgniteObjectException {
+        guard();
+
+        try {
+            return proc.metadata(proc.typeId(typeName));
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException {
+        guard();
+
+        try {
+            return proc.metadata(typeId);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgniteObjectMetadata> metadata() throws IgniteObjectException {
+        guard();
+
+        try {
+            return proc.metadata();
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /**
+     * @return Portable processor.
+     */
+    public IgniteCacheObjectProcessor processor() {
+        return proc;
+    }
+
+    /**
+     * <tt>ctx.gateway().readLock()</tt>
+     */
+    private void guard() {
+        ctx.gateway().readLock();
+    }
+
+    /**
+     * <tt>ctx.gateway().readUnlock()</tt>
+     */
+    private void unguard() {
+        ctx.gateway().readUnlock();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
deleted file mode 100644
index 5ed6505..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Collection;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * {@link IgnitePortables} implementation.
- */
-public class IgnitePortablesImpl implements IgnitePortables {
-    /** */
-    private GridKernalContext ctx;
-
-    /** */
-    private CacheObjectPortableProcessor proc;
-
-    /**
-     * @param ctx Context.
-     */
-    public IgnitePortablesImpl(GridKernalContext ctx, CacheObjectPortableProcessor proc) {
-        this.ctx = ctx;
-
-        this.proc = proc;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int typeId(String typeName) {
-        guard();
-
-        try {
-            return proc.typeId(typeName);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T toPortable(@Nullable Object obj) throws PortableException {
-        guard();
-
-        try {
-            return (T)proc.marshalToPortable(obj);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder builder(int typeId) {
-        guard();
-
-        try {
-            return proc.builder(typeId);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder builder(String typeName) {
-        guard();
-
-        try {
-            return proc.builder(typeName);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableBuilder builder(PortableObject portableObj) {
-        guard();
-
-        try {
-            return proc.builder(portableObj);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metadata(Class<?> cls) throws PortableException {
-        guard();
-
-        try {
-            return proc.metadata(proc.typeId(cls.getName()));
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metadata(String typeName) throws PortableException {
-        guard();
-
-        try {
-            return proc.metadata(proc.typeId(typeName));
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metadata(int typeId) throws PortableException {
-        guard();
-
-        try {
-            return proc.metadata(typeId);
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<PortableMetadata> metadata() throws PortableException {
-        guard();
-
-        try {
-            return proc.metadata();
-        }
-        finally {
-            unguard();
-        }
-    }
-
-    /**
-     * @return Portable processor.
-     */
-    public IgniteCacheObjectProcessor processor() {
-        return proc;
-    }
-
-    /**
-     * <tt>ctx.gateway().readLock()</tt>
-     */
-    private void guard() {
-        ctx.gateway().readLock();
-    }
-
-    /**
-     * <tt>ctx.gateway().readUnlock()</tt>
-     */
-    private void unguard() {
-        ctx.gateway().readUnlock();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 0f46517..109f0ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.platform;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
@@ -62,7 +62,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** {@inheritDoc} */
     @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
-            PortableRawReaderEx reader = platformCtx.reader(mem);
+            IgniteObjectRawReaderEx reader = platformCtx.reader(mem);
 
             if (type == OP_META) {
                 platformCtx.processMetadata(reader);
@@ -80,7 +80,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** {@inheritDoc} */
     @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
-            PortableRawReaderEx reader = platformCtx.reader(mem);
+            IgniteObjectRawReaderEx reader = platformCtx.reader(mem);
 
             return processInStreamOutObject(type, reader);
         }
@@ -104,7 +104,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
         try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             processOutStream(type, writer);
 
@@ -128,12 +128,12 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** {@inheritDoc} */
     @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
         try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
-            PortableRawReaderEx reader = platformCtx.reader(inMem);
+            IgniteObjectRawReaderEx reader = platformCtx.reader(inMem);
 
             try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
                 PlatformOutputStream out = outMem.output();
 
-                PortableRawWriterEx writer = platformCtx.writer(out);
+                IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
                 processInStreamOutStream(type, reader, writer);
 
@@ -148,12 +148,12 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     /** {@inheritDoc} */
     @Override public void inObjectStreamOutStream(int type, Object arg, long inMemPtr, long outMemPtr) throws Exception {
         try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
-            PortableRawReaderEx reader = platformCtx.reader(inMem);
+            IgniteObjectRawReaderEx reader = platformCtx.reader(inMem);
 
             try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
                 PlatformOutputStream out = outMem.output();
 
-                PortableRawWriterEx writer = platformCtx.writer(out);
+                IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
                 processInObjectStreamOutStream(type, arg, reader, writer);
 
@@ -233,7 +233,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @return Result.
      * @throws IgniteCheckedException In case of exception.
      */
-    protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
@@ -245,7 +245,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param writer Portable writer.
      * @throws IgniteCheckedException In case of exception.
      */
-    protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         throwUnsupported(type);
     }
@@ -258,7 +258,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @return Result.
      * @throws IgniteCheckedException In case of exception.
      */
-    protected Object processInStreamOutObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    protected Object processInStreamOutObject(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
@@ -271,8 +271,8 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param writer Portable writer.
      * @throws IgniteCheckedException In case of exception.
      */
-    protected void processInObjectStreamOutStream(int type, @Nullable Object arg, PortableRawReaderEx reader,
-        PortableRawWriterEx writer) throws IgniteCheckedException {
+    protected void processInObjectStreamOutStream(int type, @Nullable Object arg, IgniteObjectRawReaderEx reader,
+        IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         throwUnsupported(type);
     }
 
@@ -293,7 +293,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @param writer Portable writer.
      * @throws IgniteCheckedException In case of exception.
      */
-    protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         throwUnsupported(type);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
index a9b7d02..d4009a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContext.java
@@ -21,8 +21,8 @@ import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
@@ -71,7 +71,7 @@ public interface PlatformContext {
      * @param mem Memory.
      * @return Reader.
      */
-    public PortableRawReaderEx reader(PlatformMemory mem);
+    public IgniteObjectRawReaderEx reader(PlatformMemory mem);
 
     /**
      * Get memory reader.
@@ -79,7 +79,7 @@ public interface PlatformContext {
      * @param in Input.
      * @return Reader.
      */
-    public PortableRawReaderEx reader(PlatformInputStream in);
+    public IgniteObjectRawReaderEx reader(PlatformInputStream in);
 
     /**
      * Get memory writer.
@@ -87,7 +87,7 @@ public interface PlatformContext {
      * @param mem Memory.
      * @return Writer.
      */
-    public PortableRawWriterEx writer(PlatformMemory mem);
+    public IgniteObjectRawWriterEx writer(PlatformMemory mem);
 
     /**
      * Get memory writer.
@@ -95,7 +95,7 @@ public interface PlatformContext {
      * @param out Output.
      * @return Writer.
      */
-    public PortableRawWriterEx writer(PlatformOutputStream out);
+    public IgniteObjectRawWriterEx writer(PlatformOutputStream out);
 
     /**
      * Sends node info to native platform, if necessary.
@@ -110,7 +110,7 @@ public interface PlatformContext {
      * @param writer Writer.
      * @param node Node.
      */
-    public void writeNode(PortableRawWriterEx writer, ClusterNode node);
+    public void writeNode(IgniteObjectRawWriterEx writer, ClusterNode node);
 
     /**
      * Writes multiple node ids to a stream and sends node info to native platform, if necessary.
@@ -118,14 +118,14 @@ public interface PlatformContext {
      * @param writer Writer.
      * @param nodes Nodes.
      */
-    public void writeNodes(PortableRawWriterEx writer, Collection<ClusterNode> nodes);
+    public void writeNodes(IgniteObjectRawWriterEx writer, Collection<ClusterNode> nodes);
 
     /**
      * Process metadata from the platform.
      *
      * @param reader Reader.
      */
-    public void processMetadata(PortableRawReaderEx reader);
+    public void processMetadata(IgniteObjectRawReaderEx reader);
 
     /**
      * Write metadata for the given type ID.
@@ -133,14 +133,14 @@ public interface PlatformContext {
      * @param writer Writer.
      * @param typeId Type ID.
      */
-    public void writeMetadata(PortableRawWriterEx writer, int typeId);
+    public void writeMetadata(IgniteObjectRawWriterEx writer, int typeId);
 
     /**
      * Write all available metadata.
      *
      * @param writer Writer.
      */
-    public void writeAllMetadata(PortableRawWriterEx writer);
+    public void writeAllMetadata(IgniteObjectRawWriterEx writer);
 
     /**
      * Write cluster metrics.
@@ -148,7 +148,7 @@ public interface PlatformContext {
      * @param writer Writer.
      * @param metrics Metrics.
      */
-    public void writeClusterMetrics(PortableRawWriterEx writer, @Nullable ClusterMetrics metrics);
+    public void writeClusterMetrics(IgniteObjectRawWriterEx writer, @Nullable ClusterMetrics metrics);
 
     /**
      *
@@ -190,7 +190,7 @@ public interface PlatformContext {
      * @param writer Writer.
      * @param evt Event.
      */
-    public void writeEvent(PortableRawWriterEx writer, Event evt);
+    public void writeEvent(IgniteObjectRawWriterEx writer, Event evt);
 
     /**
      * Create local event filter.

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
index 177a732..cc7d507 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformContextImpl.java
@@ -34,9 +34,9 @@ import org.apache.ignite.events.SwapSpaceEvent;
 import org.apache.ignite.events.TaskEvent;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
-import org.apache.ignite.internal.portable.PortableMetaDataImpl;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectMetaDataImpl;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl;
@@ -70,13 +70,12 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T4;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
 import org.jetbrains.annotations.Nullable;
 
 import java.sql.Timestamp;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -171,22 +170,22 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public PortableRawReaderEx reader(PlatformMemory mem) {
+    @Override public IgniteObjectRawReaderEx reader(PlatformMemory mem) {
         return reader(mem.input());
     }
 
     /** {@inheritDoc} */
-    @Override public PortableRawReaderEx reader(PlatformInputStream in) {
+    @Override public IgniteObjectRawReaderEx reader(PlatformInputStream in) {
         return marsh.reader(in);
     }
 
     /** {@inheritDoc} */
-    @Override public PortableRawWriterEx writer(PlatformMemory mem) {
+    @Override public IgniteObjectRawWriterEx writer(PlatformMemory mem) {
         return writer(mem.output());
     }
 
     /** {@inheritDoc} */
-    @Override public PortableRawWriterEx writer(PlatformOutputStream out) {
+    @Override public IgniteObjectRawWriterEx writer(PlatformOutputStream out) {
         return marsh.writer(out);
     }
 
@@ -199,7 +198,7 @@ public class PlatformContextImpl implements PlatformContext {
         try (PlatformMemory mem0 = mem.allocate()) {
             PlatformOutputStream out = mem0.output();
 
-            PortableRawWriterEx w = writer(out);
+            IgniteObjectRawWriterEx w = writer(out);
 
             w.writeUuid(node.id());
 
@@ -234,7 +233,7 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeNode(PortableRawWriterEx writer, ClusterNode node) {
+    @Override public void writeNode(IgniteObjectRawWriterEx writer, ClusterNode node) {
         if (node == null) {
             writer.writeUuid(null);
 
@@ -247,7 +246,7 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeNodes(PortableRawWriterEx writer, Collection<ClusterNode> nodes) {
+    @Override public void writeNodes(IgniteObjectRawWriterEx writer, Collection<ClusterNode> nodes) {
         if (nodes == null) {
             writer.writeInt(-1);
 
@@ -264,7 +263,7 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeClusterMetrics(PortableRawWriterEx writer, @Nullable ClusterMetrics metrics) {
+    @Override public void writeClusterMetrics(IgniteObjectRawWriterEx writer, @Nullable ClusterMetrics metrics) {
         if (metrics == null)
             writer.writeBoolean(false);
         else {
@@ -340,17 +339,17 @@ public class PlatformContextImpl implements PlatformContext {
 
     /** {@inheritDoc} */
     @SuppressWarnings("ConstantConditions")
-    @Override public void processMetadata(PortableRawReaderEx reader) {
+    @Override public void processMetadata(IgniteObjectRawReaderEx reader) {
         Collection<T4<Integer, String, String, Map<String, Integer>>> metas = PlatformUtils.readCollection(reader,
             new PlatformReaderClosure<T4<Integer, String, String, Map<String, Integer>>>() {
-                @Override public T4<Integer, String, String, Map<String, Integer>> read(PortableRawReaderEx reader) {
+                @Override public T4<Integer, String, String, Map<String, Integer>> read(IgniteObjectRawReaderEx reader) {
                     int typeId = reader.readInt();
                     String typeName = reader.readString();
                     String affKey = reader.readString();
 
                     Map<String, Integer> fields = PlatformUtils.readMap(reader,
                         new PlatformReaderBiClosure<String, Integer>() {
-                            @Override public IgniteBiTuple<String, Integer> read(PortableRawReaderEx reader) {
+                            @Override public IgniteBiTuple<String, Integer> read(IgniteObjectRawReaderEx reader) {
                                 return F.t(reader.readString(), reader.readInt());
                             }
                         });
@@ -365,17 +364,17 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeMetadata(PortableRawWriterEx writer, int typeId) {
+    @Override public void writeMetadata(IgniteObjectRawWriterEx writer, int typeId) {
         writeMetadata0(writer, typeId, cacheObjProc.metadata(typeId));
     }
 
     /** {@inheritDoc} */
-    @Override public void writeAllMetadata(PortableRawWriterEx writer) {
-        Collection<PortableMetadata> metas = cacheObjProc.metadata();
+    @Override public void writeAllMetadata(IgniteObjectRawWriterEx writer) {
+        Collection<IgniteObjectMetadata> metas = cacheObjProc.metadata();
 
         writer.writeInt(metas.size());
 
-        for (org.apache.ignite.portable.PortableMetadata m : metas)
+        for (IgniteObjectMetadata m : metas)
             writeMetadata0(writer, cacheObjProc.typeId(m.typeName()), m);
     }
 
@@ -386,13 +385,13 @@ public class PlatformContextImpl implements PlatformContext {
      * @param typeId Type id.
      * @param meta Metadata.
      */
-    private void writeMetadata0(PortableRawWriterEx writer, int typeId, PortableMetadata meta) {
+    private void writeMetadata0(IgniteObjectRawWriterEx writer, int typeId, IgniteObjectMetadata meta) {
         if (meta == null)
             writer.writeBoolean(false);
         else {
             writer.writeBoolean(true);
 
-            Map<String, String> metaFields = ((PortableMetaDataImpl)meta).fields0();
+            Map<String, String> metaFields = ((IgniteObjectMetaDataImpl)meta).fields0();
 
             Map<String, Integer> fields = U.newHashMap(metaFields.size());
 
@@ -428,7 +427,7 @@ public class PlatformContextImpl implements PlatformContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeEvent(PortableRawWriterEx writer, Event evt) {
+    @Override public void writeEvent(IgniteObjectRawWriterEx writer, Event evt) {
         assert writer != null;
 
         if (evt == null)
@@ -566,7 +565,7 @@ public class PlatformContextImpl implements PlatformContext {
      * @param writer Writer.
      * @param evt Event.
      */
-    private void writeCommonEventData(PortableRawWriterEx writer, EventAdapter evt) {
+    private void writeCommonEventData(IgniteObjectRawWriterEx writer, EventAdapter evt) {
         PlatformUtils.writeIgniteUuid(writer, evt.id());
         writer.writeLong(evt.localOrder());
         writeNode(writer, evt.node());

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
index 825db6c..21b6baf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformExtendedException.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform;
 
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 
 /**
  * Denotes an exception which has some data to be written in a special manner.
@@ -53,5 +53,5 @@ public abstract class PlatformExtendedException extends PlatformException {
      *
      * @param writer Writer.
      */
-    public abstract void writeData(PortableRawWriterEx writer);
+    public abstract void writeData(IgniteObjectRawWriterEx writer);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index d783928..efe6212 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -26,7 +26,7 @@ import org.apache.ignite.configuration.PlatformConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteComputeImpl;
 import org.apache.ignite.internal.cluster.ClusterGroupAdapter;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
@@ -122,7 +122,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeString(ctx.gridName());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index ecdfc2c..3c071db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -28,8 +28,8 @@ import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
 import org.apache.ignite.cache.query.TextQuery;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.CacheOperationContext;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
@@ -276,7 +276,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PUT:
                 cache.put(reader.readObjectDetached(), reader.readObjectDetached());
@@ -372,7 +372,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Loads cache via localLoadCache or loadCache.
      */
-    private void loadCache0(PortableRawReaderEx reader, boolean loc) throws IgniteCheckedException {
+    private void loadCache0(IgniteObjectRawReaderEx reader, boolean loc) throws IgniteCheckedException {
         PlatformCacheEntryFilter filter = null;
 
         Object pred = reader.readObjectDetached();
@@ -389,7 +389,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, PortableRawReaderEx reader)
+    @Override protected Object processInStreamOutObject(int type, IgniteObjectRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_QRY_SQL:
@@ -432,7 +432,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      * @param reader Reader.
      * @return Arguments.
      */
-    @Nullable private Object[] readQueryArgs(PortableRawReaderEx reader) {
+    @Nullable private Object[] readQueryArgs(IgniteObjectRawReaderEx reader) {
         int cnt = reader.readInt();
 
         if (cnt > 0) {
@@ -448,7 +448,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_NAME:
                 writer.writeObject(cache.getName());
@@ -521,7 +521,7 @@ public class PlatformCache extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_GET: {
@@ -640,7 +640,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      * @param writer Writer.
      * @param results Results.
      */
-    private static void writeInvokeAllResult(PortableRawWriterEx writer, Map<Object, EntryProcessorResult> results) {
+    private static void writeInvokeAllResult(IgniteObjectRawWriterEx writer, Map<Object, EntryProcessorResult> results) {
         if (results == null) {
             writer.writeInt(-1);
 
@@ -674,7 +674,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      * @param writer Writer.
      * @param ex Exception.
      */
-    private static void writeError(PortableRawWriterEx writer, Exception ex) {
+    private static void writeError(IgniteObjectRawWriterEx writer, Exception ex) {
         if (ex.getCause() instanceof PlatformNativeException)
             writer.writeObjectDetached(((PlatformNativeException)ex.getCause()).cause());
         else {
@@ -845,7 +845,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Runs specified query.
      */
-    private PlatformQueryCursor runQuery(PortableRawReaderEx reader, Query qry) throws IgniteCheckedException {
+    private PlatformQueryCursor runQuery(IgniteObjectRawReaderEx reader, Query qry) throws IgniteCheckedException {
 
         try {
             QueryCursorEx cursor = (QueryCursorEx) cache.query(qry);
@@ -861,7 +861,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Runs specified fields query.
      */
-    private PlatformFieldsQueryCursor runFieldsQuery(PortableRawReaderEx reader, Query qry)
+    private PlatformFieldsQueryCursor runFieldsQuery(IgniteObjectRawReaderEx reader, Query qry)
         throws IgniteCheckedException {
         try {
             QueryCursorEx cursor = (QueryCursorEx) cache.query(qry);
@@ -877,7 +877,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads the query of specified type.
      */
-    private Query readInitialQuery(PortableRawReaderEx reader) throws IgniteCheckedException {
+    private Query readInitialQuery(IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         int typ = reader.readInt();
 
         switch (typ) {
@@ -900,7 +900,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads sql query.
      */
-    private Query readSqlQuery(PortableRawReaderEx reader) {
+    private Query readSqlQuery(IgniteObjectRawReaderEx reader) {
         boolean loc = reader.readBoolean();
         String sql = reader.readString();
         String typ = reader.readString();
@@ -914,7 +914,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads fields query.
      */
-    private Query readFieldsQuery(PortableRawReaderEx reader) {
+    private Query readFieldsQuery(IgniteObjectRawReaderEx reader) {
         boolean loc = reader.readBoolean();
         String sql = reader.readString();
         final int pageSize = reader.readInt();
@@ -927,7 +927,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads text query.
      */
-    private Query readTextQuery(PortableRawReaderEx reader) {
+    private Query readTextQuery(IgniteObjectRawReaderEx reader) {
         boolean loc = reader.readBoolean();
         String txt = reader.readString();
         String typ = reader.readString();
@@ -939,7 +939,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     /**
      * Reads scan query.
      */
-    private Query readScanQuery(PortableRawReaderEx reader) {
+    private Query readScanQuery(IgniteObjectRawReaderEx reader) {
         boolean loc = reader.readBoolean();
         final int pageSize = reader.readInt();
 
@@ -966,7 +966,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      */
     private static class GetAllWriter implements PlatformFutureUtils.Writer {
         /** <inheritDoc /> */
-        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+        @Override public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err) {
             assert obj instanceof Map;
 
             PlatformUtils.writeNullableMap(writer, (Map) obj);
@@ -983,7 +983,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      */
     private static class EntryProcessorInvokeWriter implements PlatformFutureUtils.Writer {
         /** <inheritDoc /> */
-        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+        @Override public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err) {
             if (err == null) {
                 writer.writeBoolean(false);  // No error.
 
@@ -1007,7 +1007,7 @@ public class PlatformCache extends PlatformAbstractTarget {
      */
     private static class EntryProcessorInvokeAllWriter implements PlatformFutureUtils.Writer {
         /** <inheritDoc /> */
-        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+        @Override public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err) {
             writeInvokeAllResult(writer, (Map)obj);
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
index 5f8ec8f..1dfb428 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryFilterImpl.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.cache;
 
 import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -58,7 +58,7 @@ public class PlatformCacheEntryFilterImpl extends PlatformAbstractPredicate impl
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(k);
             writer.writeObject(v);
@@ -94,7 +94,7 @@ public class PlatformCacheEntryFilterImpl extends PlatformAbstractPredicate impl
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(pred);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
index f59a63f..4be6d7e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheEntryProcessorImpl.java
@@ -26,8 +26,8 @@ import javax.cache.processor.MutableEntry;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
@@ -119,7 +119,7 @@ public class PlatformCacheEntryProcessorImpl implements PlatformCacheEntryProces
         try (PlatformMemory outMem = ctx.memory().allocate()) {
             PlatformOutputStream out = outMem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writeEntryAndProcessor(entry, writer);
 
@@ -132,7 +132,7 @@ public class PlatformCacheEntryProcessorImpl implements PlatformCacheEntryProces
 
                 in.synchronize();
 
-                PortableRawReaderEx reader = ctx.reader(in);
+                IgniteObjectRawReaderEx reader = ctx.reader(in);
 
                 return readResultAndUpdateEntry(ctx, entry, reader);
             }
@@ -145,7 +145,7 @@ public class PlatformCacheEntryProcessorImpl implements PlatformCacheEntryProces
      * @param entry Entry to process.
      * @param writer Writer.
      */
-    private void writeEntryAndProcessor(MutableEntry entry, PortableRawWriterEx writer) {
+    private void writeEntryAndProcessor(MutableEntry entry, IgniteObjectRawWriterEx writer) {
         writer.writeObject(entry.getKey());
         writer.writeObject(entry.getValue());
 
@@ -170,7 +170,7 @@ public class PlatformCacheEntryProcessorImpl implements PlatformCacheEntryProces
      * @throws javax.cache.processor.EntryProcessorException If processing has failed in user code.
      */
     @SuppressWarnings("unchecked")
-    private Object readResultAndUpdateEntry(PlatformContext ctx, MutableEntry entry, PortableRawReaderEx reader) {
+    private Object readResultAndUpdateEntry(PlatformContext ctx, MutableEntry entry, IgniteObjectRawReaderEx reader) {
         byte state = reader.readByte();
 
         switch (state) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
index 78ca683..d4c5bfa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.platform.cache;
 import java.util.Iterator;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 
@@ -47,7 +47,7 @@ public class PlatformCacheIterator extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_NEXT:
                 if (iter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
index ef17a06..4323ae0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCachePartialUpdateException.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.cache;
 
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
@@ -49,7 +49,7 @@ public class PlatformCachePartialUpdateException extends PlatformExtendedExcepti
     }
 
     /** {@inheritDoc} */
-    @Override public void writeData(PortableRawWriterEx writer) {
+    @Override public void writeData(IgniteObjectRawWriterEx writer) {
         Collection keys = ((CachePartialUpdateCheckedException)getCause()).failedKeys();
 
         writer.writeBoolean(keepPortable);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
index 0d2098b..b628855 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -25,8 +25,8 @@ import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -114,7 +114,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PARTITION:
                 return aff.partition(reader.readObjectDetached());
@@ -165,7 +165,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_PRIMARY_PARTITIONS: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
index 6c2c873..737e79f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.cache.query;
 
 import java.util.Iterator;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
@@ -62,7 +62,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, final PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, final IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_BATCH: {
                 assert iter != null : "iterator() has not been called";
@@ -157,7 +157,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
      * @param writer Writer.
      * @param val Value.
      */
-    protected abstract void write(PortableRawWriterEx writer, T val);
+    protected abstract void write(IgniteObjectRawWriterEx writer, T val);
 
     /**
      * Query cursor consumer.
@@ -167,7 +167,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
         private final PlatformAbstractQueryCursor<T> cursor;
 
         /** Writer. */
-        private final PortableRawWriterEx writer;
+        private final IgniteObjectRawWriterEx writer;
 
         /** Count. */
         private int cnt;
@@ -177,7 +177,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
          *
          * @param writer Writer.
          */
-        public Consumer(PlatformAbstractQueryCursor<T> cursor, PortableRawWriterEx writer) {
+        public Consumer(PlatformAbstractQueryCursor<T> cursor, IgniteObjectRawWriterEx writer) {
             this.cursor = cursor;
             this.writer = writer;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
index 71aa38c..0100a59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryRemoteFilter.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.cache.query;
 
 import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -115,7 +115,7 @@ public class PlatformContinuousQueryRemoteFilter implements PlatformContinuousQu
             try (PlatformMemory mem = ctx.memory().allocate()) {
                 PlatformOutputStream out = mem.output();
 
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 writer.writeObject(filter);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
index 44a4f14..6f8f6dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformFieldsQueryCursor.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.cache.query;
 
 import java.util.List;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 
@@ -38,7 +38,7 @@ public class PlatformFieldsQueryCursor extends PlatformAbstractQueryCursor<List<
     }
 
     /** {@inheritDoc} */
-    @Override protected void write(PortableRawWriterEx writer, List vals) {
+    @Override protected void write(IgniteObjectRawWriterEx writer, List vals) {
         assert vals != null;
 
         writer.writeInt(vals.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
index 410e4de..6429663 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformQueryCursor.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.cache.query;
 
 import javax.cache.Cache;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.cache.query.QueryCursorEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 
@@ -38,7 +38,7 @@ public class PlatformQueryCursor extends PlatformAbstractQueryCursor<Cache.Entry
     }
 
     /** {@inheritDoc} */
-    @Override protected void write(PortableRawWriterEx writer, Cache.Entry val) {
+    @Override protected void write(IgniteObjectRawWriterEx writer, Cache.Entry val) {
         writer.writeObjectDetached(val.getKey());
         writer.writeObjectDetached(val.getValue());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
index a741f0f..42f9b86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/store/PlatformCacheStoreCallback.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.cache.store;
 
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 
@@ -45,7 +45,7 @@ public abstract class PlatformCacheStoreCallback {
     public void invoke(long memPtr) {
         if (memPtr > 0) {
             try (PlatformMemory mem = ctx.memory().get(memPtr)) {
-                PortableRawReaderEx reader = ctx.reader(mem);
+                IgniteObjectRawReaderEx reader = ctx.reader(mem);
 
                 invoke0(reader);
             }
@@ -57,5 +57,5 @@ public abstract class PlatformCacheStoreCallback {
      *
      * @param reader Reader.
      */
-    protected abstract void invoke0(PortableRawReaderEx reader);
+    protected abstract void invoke0(IgniteObjectRawReaderEx reader);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index a1c8516..f839653 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -24,8 +24,8 @@ import org.apache.ignite.IgniteCluster;
 import org.apache.ignite.cluster.ClusterMetrics;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.cluster.ClusterGroupEx;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -94,7 +94,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings("deprecation")
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS:
                 platformCtx.writeClusterMetrics(writer, prj.metrics());
@@ -113,7 +113,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ConstantConditions", "deprecation"})
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS_FILTERED: {
@@ -194,7 +194,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PING_NODE:
                 return pingNode(reader.readUuid()) ? TRUE : FALSE;
@@ -205,7 +205,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected Object processInStreamOutObject(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_FOR_NODE_IDS: {
                 Collection<UUID> ids = PlatformUtils.readCollection(reader);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
index 5ba9a85..022aad4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterNodeFilterImpl.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.cluster;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -56,7 +56,7 @@ public class PlatformClusterNodeFilterImpl extends PlatformAbstractPredicate imp
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(pred);
             ctx.writeNode(writer, clusterNode);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
index bf9d9e4..7051164 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractJob.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.platform.compute;
 import java.io.Externalizable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -105,7 +105,7 @@ public abstract class PlatformAbstractJob implements PlatformJob, Externalizable
             try (PlatformMemory mem = ctx.memory().allocate()) {
                 PlatformOutputStream out = mem.output();
 
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 writer.writeObject(job);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
index b17dd97..740c901 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformAbstractTask.java
@@ -24,7 +24,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTask;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -86,7 +86,7 @@ public abstract class PlatformAbstractTask implements ComputeTask<Object, Void>
                 try (PlatformMemory mem = ctx.memory().allocate()) {
                     PlatformOutputStream out = mem.output();
 
-                    PortableRawWriterEx writer = ctx.writer(out);
+                    IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                     writer.writeUuid(res.getNode().id());
                     writer.writeBoolean(res.isCancelled());
@@ -151,7 +151,7 @@ public abstract class PlatformAbstractTask implements ComputeTask<Object, Void>
                 try (PlatformMemory mem = ctx.memory().allocate()) {
                     PlatformOutputStream out = mem.output();
 
-                    PortableRawWriterEx writer = ctx.writer(out);
+                    IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                     if (e0 == null) {
                         writer.writeBoolean(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
index 9bd7d60..281dc52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformClosureJob.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -69,7 +69,7 @@ public class PlatformClosureJob extends PlatformAbstractJob {
 
                 in.synchronize();
 
-                PortableRawReaderEx reader = ctx.reader(in);
+                IgniteObjectRawReaderEx reader = ctx.reader(in);
 
                 return PlatformUtils.readInvocationResult(ctx, reader);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 638b4b1..35e0051 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -25,15 +25,15 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.internal.IgniteComputeImpl;
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.portable.PortableObjectImpl;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectImpl;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.util.typedef.C1;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_SUBGRID;
 
@@ -75,7 +75,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_UNICAST:
                 processClosures(reader.readLong(), reader, false, false);
@@ -104,7 +104,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param reader Reader.
      * @param broadcast broadcast flag.
      */
-    private void processClosures(long taskPtr, PortableRawReaderEx reader, boolean broadcast, boolean affinity) {
+    private void processClosures(long taskPtr, IgniteObjectRawReaderEx reader, boolean broadcast, boolean affinity) {
         PlatformAbstractTask task;
 
         int size = reader.readInt();
@@ -165,12 +165,12 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param reader Reader.
      * @return Closure job.
      */
-    private PlatformJob nextClosureJob(PlatformAbstractTask task, PortableRawReaderEx reader) {
+    private PlatformJob nextClosureJob(PlatformAbstractTask task, IgniteObjectRawReaderEx reader) {
         return platformCtx.createClosureJob(task, reader.readLong(), reader.readObjectDetached());
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_EXEC:
@@ -256,7 +256,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param reader Reader.
      * @return Task result.
      */
-    protected Object executeJavaTask(PortableRawReaderEx reader, boolean async) {
+    protected Object executeJavaTask(IgniteObjectRawReaderEx reader, boolean async) {
         String taskName = reader.readString();
         boolean keepPortable = reader.readBoolean();
         Object arg = reader.readObjectDetached();
@@ -268,8 +268,8 @@ public class PlatformCompute extends PlatformAbstractTarget {
         if (async)
             compute0 = compute0.withAsync();
 
-        if (!keepPortable && arg instanceof PortableObjectImpl)
-            arg = ((PortableObject)arg).deserialize();
+        if (!keepPortable && arg instanceof IgniteObjectImpl)
+            arg = ((IgniteObject)arg).deserialize();
 
         Object res = compute0.execute(taskName, arg);
 
@@ -304,7 +304,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param reader Reader.
      * @return Node IDs.
      */
-    protected Collection<UUID> readNodeIds(PortableRawReaderEx reader) {
+    protected Collection<UUID> readNodeIds(IgniteObjectRawReaderEx reader) {
         if (reader.readBoolean()) {
             int len = reader.readInt();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
index cfed735..5fc9078 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullJob.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformProcessor;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
@@ -120,7 +120,7 @@ public class PlatformFullJob extends PlatformAbstractJob {
 
                     in.synchronize();
 
-                    PortableRawReaderEx reader = ctx.reader(in);
+                    IgniteObjectRawReaderEx reader = ctx.reader(in);
 
                     return PlatformUtils.readInvocationResult(ctx, reader);
                 }
@@ -209,7 +209,7 @@ public class PlatformFullJob extends PlatformAbstractJob {
 
             in.synchronize();
 
-            PortableRawReaderEx reader = ctx.reader(in);
+            IgniteObjectRawReaderEx reader = ctx.reader(in);
 
             if (res)
                 job = reader.readObjectDetached();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
index b96d445..1c943e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformFullTask.java
@@ -27,8 +27,8 @@ import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeTaskNoResultCache;
 import org.apache.ignite.internal.IgniteComputeImpl;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -83,7 +83,7 @@ public final class PlatformFullTask extends PlatformAbstractTask {
             try (PlatformMemory outMem = memMgr.allocate()) {
                 PlatformOutputStream out = outMem.output();
 
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 write(writer, nodes, subgrid);
 
@@ -96,7 +96,7 @@ public final class PlatformFullTask extends PlatformAbstractTask {
 
                     in.synchronize();
 
-                    PortableRawReaderEx reader = ctx.reader(in);
+                    IgniteObjectRawReaderEx reader = ctx.reader(in);
 
                     return read(reader, nodes);
                 }
@@ -114,7 +114,7 @@ public final class PlatformFullTask extends PlatformAbstractTask {
      * @param nodes Current topology nodes.
      * @param subgrid Subgrid.
      */
-    private void write(PortableRawWriterEx writer, Collection<ClusterNode> nodes, List<ClusterNode> subgrid) {
+    private void write(IgniteObjectRawWriterEx writer, Collection<ClusterNode> nodes, List<ClusterNode> subgrid) {
         GridDiscoveryManager discoMgr = ctx.kernalContext().discovery();
 
         long curTopVer = discoMgr.topologyVersion();
@@ -145,7 +145,7 @@ public final class PlatformFullTask extends PlatformAbstractTask {
      * @param nodes Current topology nodes.
      * @return Map result.
      */
-    private Map<ComputeJob, ClusterNode> read(PortableRawReaderEx reader, Collection<ClusterNode> nodes) {
+    private Map<ComputeJob, ClusterNode> read(IgniteObjectRawReaderEx reader, Collection<ClusterNode> nodes) {
         if (reader.readBoolean()) {
             if (!reader.readBoolean())
                 return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index ef64ef9..776f26b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -24,7 +24,7 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
@@ -87,7 +87,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc}  */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader) throws IgniteCheckedException {
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_UPDATE:
                 int plc = reader.readInt();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
index 92250c0..e814b03 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.platform.datastreamer;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
@@ -76,7 +76,7 @@ public class PlatformStreamReceiverImpl extends PlatformAbstractPredicate implem
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(pred);
 


[04/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridIgniteObjectMarshalerAwareTestClass.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridIgniteObjectMarshalerAwareTestClass.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridIgniteObjectMarshalerAwareTestClass.java
new file mode 100644
index 0000000..9fe4242
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridIgniteObjectMarshalerAwareTestClass.java
@@ -0,0 +1,67 @@
+/*
+ * 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.mutabletest;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectRawReader;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class GridIgniteObjectMarshalerAwareTestClass implements IgniteObjectMarshalAware {
+    /** */
+    public String s;
+
+    /** */
+    public String sRaw;
+
+    /** {@inheritDoc} */
+    @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+        writer.writeString("s", s);
+
+        IgniteObjectRawWriter raw = writer.rawWriter();
+
+        raw.writeString(sRaw);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+        s = reader.readString("s");
+
+        IgniteObjectRawReader raw = reader.rawReader();
+
+        sRaw = raw.readString();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("FloatingPointEquality")
+    @Override public boolean equals(Object other) {
+        return this == other || GridTestUtils.deepEquals(this, other);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridIgniteObjectMarshalerAwareTestClass.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
deleted file mode 100644
index 3244331..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableMarshalerAwareTestClass.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable.mutabletest;
-
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
-import org.apache.ignite.testframework.GridTestUtils;
-
-/**
- *
- */
-public class GridPortableMarshalerAwareTestClass implements PortableMarshalAware {
-    /** */
-    public String s;
-
-    /** */
-    public String sRaw;
-
-    /** {@inheritDoc} */
-    @Override public void writePortable(PortableWriter writer) throws PortableException {
-        writer.writeString("s", s);
-
-        PortableRawWriter raw = writer.rawWriter();
-
-        raw.writeString(sRaw);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readPortable(PortableReader reader) throws PortableException {
-        s = reader.readString("s");
-
-        PortableRawReader raw = reader.rawReader();
-
-        sRaw = raw.readString();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("FloatingPointEquality")
-    @Override public boolean equals(Object other) {
-        return this == other || GridTestUtils.deepEquals(this, other);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridPortableMarshalerAwareTestClass.class, this);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
index a2ae71f..af9be0b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/mutabletest/GridPortableTestClasses.java
@@ -32,7 +32,7 @@ import java.util.Map;
 import java.util.TreeMap;
 import java.util.UUID;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.igniteobject.IgniteObject;
 
 /**
  *
@@ -106,7 +106,7 @@ public class GridPortableTestClasses {
      */
     public static class TestObjectPlainPortable {
         /** */
-        public PortableObject plainPortable;
+        public IgniteObject plainPortable;
 
         /**
          *
@@ -118,7 +118,7 @@ public class GridPortableTestClasses {
         /**
          * @param plainPortable Object.
          */
-        public TestObjectPlainPortable(PortableObject plainPortable) {
+        public TestObjectPlainPortable(IgniteObject plainPortable) {
             this.plainPortable = plainPortable;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataMultinodeTest.java
new file mode 100644
index 0000000..e86a160
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataMultinodeTest.java
@@ -0,0 +1,295 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteObjects;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.eclipse.jetty.util.ConcurrentHashSet;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class GridCacheClientNodeIgniteObjectMetadataMultinodeTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
+
+        cfg.setMarshaller(new PortableMarshaller());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientMetadataInitialization() throws Exception {
+        startGrids(2);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final ConcurrentHashSet<String> allTypes = new ConcurrentHashSet<>();
+
+        IgniteInternalFuture<?> fut;
+
+        try {
+            // Update portable metadata concurrently with client nodes start.
+            fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    IgniteObjects portables = ignite(0).portables();
+
+                    IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepBinary();
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    for (int i = 0; i < 1000; i++) {
+                        log.info("Iteration: " + i);
+
+                        String type = "portable-type-" + i;
+
+                        allTypes.add(type);
+
+                        for (int f = 0; f < 10; f++) {
+                            IgniteObjectBuilder builder = portables.builder(type);
+
+                            String fieldName = "f" + f;
+
+                            builder.setField(fieldName, i);
+
+                            cache.put(rnd.nextInt(0, 100_000), builder.build());
+
+                            if (f % 100 == 0)
+                                log.info("Put iteration: " + f);
+                        }
+
+                        if (stop.get())
+                            break;
+                    }
+
+                    return null;
+                }
+            }, 5, "update-thread");
+        }
+        finally {
+            stop.set(true);
+        }
+
+        client = true;
+
+        startGridsMultiThreaded(2, 5);
+
+        fut.get();
+
+        assertFalse(allTypes.isEmpty());
+
+        log.info("Expected portable types: " + allTypes.size());
+
+        assertEquals(7, ignite(0).cluster().nodes().size());
+
+        for (int i = 0; i < 7; i++) {
+            log.info("Check metadata on node: " + i);
+
+            boolean client = i > 1;
+
+            assertEquals((Object)client, ignite(i).configuration().isClientMode());
+
+            IgniteObjects portables = ignite(i).portables();
+
+            Collection<IgniteObjectMetadata> metaCol = portables.metadata();
+
+            assertEquals(allTypes.size(), metaCol.size());
+
+            Set<String> names = new HashSet<>();
+
+            for (IgniteObjectMetadata meta : metaCol) {
+                assertTrue(names.add(meta.typeName()));
+
+                assertNull(meta.affinityKeyFieldName());
+
+                assertEquals(10, meta.fields().size());
+            }
+
+            assertEquals(allTypes.size(), names.size());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFailoverOnStart() throws Exception {
+        startGrids(4);
+
+        IgniteObjects portables = ignite(0).portables();
+
+        IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepBinary();
+
+        for (int i = 0; i < 1000; i++) {
+            IgniteObjectBuilder builder = portables.builder("type-" + i);
+
+            builder.setField("f0", i);
+
+            cache.put(i, builder.build());
+        }
+
+        client = true;
+
+        final CyclicBarrier barrier = new CyclicBarrier(6);
+
+        final AtomicInteger startIdx = new AtomicInteger(4);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                barrier.await();
+
+                Ignite ignite = startGrid(startIdx.getAndIncrement());
+
+                assertTrue(ignite.configuration().isClientMode());
+
+                log.info("Started node: " + ignite.name());
+
+                return null;
+            }
+        }, 5, "start-thread");
+
+        barrier.await();
+
+        U.sleep(ThreadLocalRandom.current().nextInt(10, 100));
+
+        for (int i = 0; i < 3; i++)
+            stopGrid(i);
+
+        fut.get();
+
+        assertEquals(6, ignite(3).cluster().nodes().size());
+
+        for (int i = 3; i < 7; i++) {
+            log.info("Check metadata on node: " + i);
+
+            boolean client = i > 3;
+
+            assertEquals((Object) client, ignite(i).configuration().isClientMode());
+
+            portables = ignite(i).portables();
+
+            final IgniteObjects p0 = portables;
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    Collection<IgniteObjectMetadata> metaCol = p0.metadata();
+
+                    return metaCol.size() == 1000;
+                }
+            }, getTestTimeout());
+
+            Collection<IgniteObjectMetadata> metaCol = portables.metadata();
+
+            assertEquals(1000, metaCol.size());
+
+            Set<String> names = new HashSet<>();
+
+            for (IgniteObjectMetadata meta : metaCol) {
+                assertTrue(names.add(meta.typeName()));
+
+                assertNull(meta.affinityKeyFieldName());
+
+                assertEquals(1, meta.fields().size());
+            }
+
+            assertEquals(1000, names.size());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientStartsFirst() throws Exception {
+        client = true;
+
+        Ignite ignite0 = startGrid(0);
+
+        assertTrue(ignite0.configuration().isClientMode());
+
+        client = false;
+
+        Ignite ignite1 = startGrid(1);
+
+        assertFalse(ignite1.configuration().isClientMode());
+
+        IgniteObjects portables = ignite(1).portables();
+
+        IgniteCache<Object, Object> cache = ignite(1).cache(null).withKeepBinary();
+
+        for (int i = 0; i < 100; i++) {
+            IgniteObjectBuilder builder = portables.builder("type-" + i);
+
+            builder.setField("f0", i);
+
+            cache.put(i, builder.build());
+        }
+
+        assertEquals(100, ignite(0).portables().metadata().size());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataTest.java
new file mode 100644
index 0000000..addf1f8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodeIgniteObjectMetadataTest.java
@@ -0,0 +1,290 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.util.Arrays;
+import java.util.Collection;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.affinity.Affinity;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class GridCacheClientNodeIgniteObjectMetadataTest extends GridCacheAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return CacheMode.PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
+
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration();
+
+        typeCfg.setClassName(TestObject1.class.getName());
+
+        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject1.class.getName(), "val2");
+
+        cfg.setCacheKeyCfg(keyCfg);
+
+        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
+
+        if (gridName.equals(getTestGridName(gridCount() - 1)))
+            cfg.setClientMode(true);
+
+        cfg.setMarshaller(marsh);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableMetadataOnClient() throws Exception {
+        Ignite ignite0 = ignite(gridCount() - 1);
+
+        assertTrue(ignite0.configuration().isClientMode());
+
+        Ignite ignite1 = ignite(0);
+
+        assertFalse(ignite1.configuration().isClientMode());
+
+        Affinity<Object> aff0 = ignite0.affinity(null);
+        Affinity<Object> aff1 = ignite1.affinity(null);
+
+        for (int i = 0 ; i < 100; i++) {
+            TestObject1 obj1 = new TestObject1(i, i + 1);
+
+            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj1),
+                aff0.mapKeyToPrimaryAndBackups(obj1));
+
+            TestObject2 obj2 = new TestObject2(i, i + 1);
+
+            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj2),
+                aff0.mapKeyToPrimaryAndBackups(obj2));
+        }
+
+        {
+            IgniteObjectBuilder builder = ignite0.portables().builder("TestObject3");
+
+            builder.setField("f1", 1);
+
+            ignite0.cache(null).put(0, builder.build());
+
+            IgniteCache<Integer, IgniteObject> cache = ignite0.cache(null).withKeepBinary();
+
+            IgniteObject obj = cache.get(0);
+
+            IgniteObjectMetadata meta = obj.metaData();
+
+            assertNotNull(meta);
+            assertEquals(1, meta.fields().size());
+
+            meta = ignite0.portables().metadata(TestObject1.class);
+
+            assertNotNull(meta);
+            assertEquals("val2", meta.affinityKeyFieldName());
+
+            meta = ignite0.portables().metadata(TestObject2.class);
+
+            assertNotNull(meta);
+            assertNull(meta.affinityKeyFieldName());
+        }
+
+        {
+            IgniteObjectBuilder builder = ignite1.portables().builder("TestObject3");
+
+            builder.setField("f2", 2);
+
+            ignite1.cache(null).put(1, builder.build());
+
+            IgniteCache<Integer, IgniteObject> cache = ignite1.cache(null).withKeepBinary();
+
+            IgniteObject obj = cache.get(0);
+
+            IgniteObjectMetadata meta = obj.metaData();
+
+            assertNotNull(meta);
+            assertEquals(2, meta.fields().size());
+
+            meta = ignite1.portables().metadata(TestObject1.class);
+
+            assertNotNull(meta);
+            assertEquals("val2", meta.affinityKeyFieldName());
+
+            meta = ignite1.portables().metadata(TestObject2.class);
+
+            assertNotNull(meta);
+            assertNull(meta.affinityKeyFieldName());
+        }
+
+        IgniteObjectMetadata meta = ignite0.portables().metadata("TestObject3");
+
+        assertNotNull(meta);
+        assertEquals(2, meta.fields().size());
+
+        IgniteCache<Integer, IgniteObject> cache = ignite0.cache(null).withKeepBinary();
+
+        IgniteObject obj = cache.get(1);
+
+        assertEquals(Integer.valueOf(2), obj.field("f2"));
+        assertNull(obj.field("f1"));
+
+        meta = obj.metaData();
+
+        assertNotNull(meta);
+        assertEquals(2, meta.fields().size());
+
+        Collection<IgniteObjectMetadata> meta1 = ignite1.portables().metadata();
+        Collection<IgniteObjectMetadata> meta2 = ignite1.portables().metadata();
+
+        assertEquals(meta1.size(), meta2.size());
+
+        for (IgniteObjectMetadata m1 : meta1) {
+            boolean found = false;
+
+            for (IgniteObjectMetadata m2 : meta1) {
+                if (m1.typeName().equals(m2.typeName())) {
+                    assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName());
+                    assertEquals(m1.fields(), m2.fields());
+
+                    found = true;
+
+                    break;
+                }
+            }
+
+            assertTrue(found);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestObject1 {
+        /** */
+        private int val1;
+
+        /** */
+        private int val2;
+
+        /**
+         * @param val1 Value 1.
+         * @param val2 Value 2.
+         */
+        public TestObject1(int val1, int val2) {
+            this.val1 = val1;
+            this.val2 = val2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestObject1 that = (TestObject1)o;
+
+            return val1 == that.val1;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val1;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestObject2 {
+        /** */
+        private int val1;
+
+        /** */
+        private int val2;
+
+        /**
+         * @param val1 Value 1.
+         * @param val2 Value 2.
+         */
+        public TestObject2(int val1, int val2) {
+            this.val1 = val1;
+            this.val2 = val2;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestObject2 that = (TestObject2)o;
+
+            return val2 == that.val2;
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val2;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
deleted file mode 100644
index 1ba3d4d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataMultinodeTest.java
+++ /dev/null
@@ -1,295 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.util.lang.GridAbsPredicate;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.eclipse.jetty.util.ConcurrentHashSet;
-
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- *
- */
-public class GridCacheClientNodePortableMetadataMultinodeTest extends GridCommonAbstractTest {
-    /** */
-    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private boolean client;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
-
-        cfg.setMarshaller(new PortableMarshaller());
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setWriteSynchronizationMode(FULL_SYNC);
-
-        cfg.setCacheConfiguration(ccfg);
-
-        cfg.setClientMode(client);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        super.afterTest();
-
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientMetadataInitialization() throws Exception {
-        startGrids(2);
-
-        final AtomicBoolean stop = new AtomicBoolean();
-
-        final ConcurrentHashSet<String> allTypes = new ConcurrentHashSet<>();
-
-        IgniteInternalFuture<?> fut;
-
-        try {
-            // Update portable metadata concurrently with client nodes start.
-            fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    IgnitePortables portables = ignite(0).portables();
-
-                    IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepPortable();
-
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    for (int i = 0; i < 1000; i++) {
-                        log.info("Iteration: " + i);
-
-                        String type = "portable-type-" + i;
-
-                        allTypes.add(type);
-
-                        for (int f = 0; f < 10; f++) {
-                            PortableBuilder builder = portables.builder(type);
-
-                            String fieldName = "f" + f;
-
-                            builder.setField(fieldName, i);
-
-                            cache.put(rnd.nextInt(0, 100_000), builder.build());
-
-                            if (f % 100 == 0)
-                                log.info("Put iteration: " + f);
-                        }
-
-                        if (stop.get())
-                            break;
-                    }
-
-                    return null;
-                }
-            }, 5, "update-thread");
-        }
-        finally {
-            stop.set(true);
-        }
-
-        client = true;
-
-        startGridsMultiThreaded(2, 5);
-
-        fut.get();
-
-        assertFalse(allTypes.isEmpty());
-
-        log.info("Expected portable types: " + allTypes.size());
-
-        assertEquals(7, ignite(0).cluster().nodes().size());
-
-        for (int i = 0; i < 7; i++) {
-            log.info("Check metadata on node: " + i);
-
-            boolean client = i > 1;
-
-            assertEquals((Object)client, ignite(i).configuration().isClientMode());
-
-            IgnitePortables portables = ignite(i).portables();
-
-            Collection<PortableMetadata> metaCol = portables.metadata();
-
-            assertEquals(allTypes.size(), metaCol.size());
-
-            Set<String> names = new HashSet<>();
-
-            for (PortableMetadata meta : metaCol) {
-                assertTrue(names.add(meta.typeName()));
-
-                assertNull(meta.affinityKeyFieldName());
-
-                assertEquals(10, meta.fields().size());
-            }
-
-            assertEquals(allTypes.size(), names.size());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFailoverOnStart() throws Exception {
-        startGrids(4);
-
-        IgnitePortables portables = ignite(0).portables();
-
-        IgniteCache<Object, Object> cache = ignite(0).cache(null).withKeepPortable();
-
-        for (int i = 0; i < 1000; i++) {
-            PortableBuilder builder = portables.builder("type-" + i);
-
-            builder.setField("f0", i);
-
-            cache.put(i, builder.build());
-        }
-
-        client = true;
-
-        final CyclicBarrier barrier = new CyclicBarrier(6);
-
-        final AtomicInteger startIdx = new AtomicInteger(4);
-
-        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                barrier.await();
-
-                Ignite ignite = startGrid(startIdx.getAndIncrement());
-
-                assertTrue(ignite.configuration().isClientMode());
-
-                log.info("Started node: " + ignite.name());
-
-                return null;
-            }
-        }, 5, "start-thread");
-
-        barrier.await();
-
-        U.sleep(ThreadLocalRandom.current().nextInt(10, 100));
-
-        for (int i = 0; i < 3; i++)
-            stopGrid(i);
-
-        fut.get();
-
-        assertEquals(6, ignite(3).cluster().nodes().size());
-
-        for (int i = 3; i < 7; i++) {
-            log.info("Check metadata on node: " + i);
-
-            boolean client = i > 3;
-
-            assertEquals((Object) client, ignite(i).configuration().isClientMode());
-
-            portables = ignite(i).portables();
-
-            final IgnitePortables p0 = portables;
-
-            GridTestUtils.waitForCondition(new GridAbsPredicate() {
-                @Override public boolean apply() {
-                    Collection<PortableMetadata> metaCol = p0.metadata();
-
-                    return metaCol.size() == 1000;
-                }
-            }, getTestTimeout());
-
-            Collection<PortableMetadata> metaCol = portables.metadata();
-
-            assertEquals(1000, metaCol.size());
-
-            Set<String> names = new HashSet<>();
-
-            for (PortableMetadata meta : metaCol) {
-                assertTrue(names.add(meta.typeName()));
-
-                assertNull(meta.affinityKeyFieldName());
-
-                assertEquals(1, meta.fields().size());
-            }
-
-            assertEquals(1000, names.size());
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientStartsFirst() throws Exception {
-        client = true;
-
-        Ignite ignite0 = startGrid(0);
-
-        assertTrue(ignite0.configuration().isClientMode());
-
-        client = false;
-
-        Ignite ignite1 = startGrid(1);
-
-        assertFalse(ignite1.configuration().isClientMode());
-
-        IgnitePortables portables = ignite(1).portables();
-
-        IgniteCache<Object, Object> cache = ignite(1).cache(null).withKeepPortable();
-
-        for (int i = 0; i < 100; i++) {
-            PortableBuilder builder = portables.builder("type-" + i);
-
-            builder.setField("f0", i);
-
-            cache.put(i, builder.build());
-        }
-
-        assertEquals(100, ignite(0).portables().metadata().size());
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
deleted file mode 100644
index a66d940..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheClientNodePortableMetadataTest.java
+++ /dev/null
@@ -1,286 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.portable;
-
-import java.util.Arrays;
-import java.util.Collection;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-
-/**
- *
- */
-public class GridCacheClientNodePortableMetadataTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(TestObject1.class.getName(), TestObject2.class.getName()));
-
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
-
-        typeCfg.setClassName(TestObject1.class.getName());
-        typeCfg.setAffinityKeyFieldName("val2");
-
-        marsh.setTypeConfigurations(Arrays.asList(typeCfg));
-
-        if (gridName.equals(getTestGridName(gridCount() - 1)))
-            cfg.setClientMode(true);
-
-        cfg.setMarshaller(marsh);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableMetadataOnClient() throws Exception {
-        Ignite ignite0 = ignite(gridCount() - 1);
-
-        assertTrue(ignite0.configuration().isClientMode());
-
-        Ignite ignite1 = ignite(0);
-
-        assertFalse(ignite1.configuration().isClientMode());
-
-        Affinity<Object> aff0 = ignite0.affinity(null);
-        Affinity<Object> aff1 = ignite1.affinity(null);
-
-        for (int i = 0 ; i < 100; i++) {
-            TestObject1 obj1 = new TestObject1(i, i + 1);
-
-            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj1),
-                aff0.mapKeyToPrimaryAndBackups(obj1));
-
-            TestObject2 obj2 = new TestObject2(i, i + 1);
-
-            assertEquals(aff1.mapKeyToPrimaryAndBackups(obj2),
-                aff0.mapKeyToPrimaryAndBackups(obj2));
-        }
-
-        {
-            PortableBuilder builder = ignite0.portables().builder("TestObject3");
-
-            builder.setField("f1", 1);
-
-            ignite0.cache(null).put(0, builder.build());
-
-            IgniteCache<Integer, PortableObject> cache = ignite0.cache(null).withKeepPortable();
-
-            PortableObject obj = cache.get(0);
-
-            PortableMetadata meta = obj.metaData();
-
-            assertNotNull(meta);
-            assertEquals(1, meta.fields().size());
-
-            meta = ignite0.portables().metadata(TestObject1.class);
-
-            assertNotNull(meta);
-            assertEquals("val2", meta.affinityKeyFieldName());
-
-            meta = ignite0.portables().metadata(TestObject2.class);
-
-            assertNotNull(meta);
-            assertNull(meta.affinityKeyFieldName());
-        }
-
-        {
-            PortableBuilder builder = ignite1.portables().builder("TestObject3");
-
-            builder.setField("f2", 2);
-
-            ignite1.cache(null).put(1, builder.build());
-
-            IgniteCache<Integer, PortableObject> cache = ignite1.cache(null).withKeepPortable();
-
-            PortableObject obj = cache.get(0);
-
-            PortableMetadata meta = obj.metaData();
-
-            assertNotNull(meta);
-            assertEquals(2, meta.fields().size());
-
-            meta = ignite1.portables().metadata(TestObject1.class);
-
-            assertNotNull(meta);
-            assertEquals("val2", meta.affinityKeyFieldName());
-
-            meta = ignite1.portables().metadata(TestObject2.class);
-
-            assertNotNull(meta);
-            assertNull(meta.affinityKeyFieldName());
-        }
-
-        PortableMetadata meta = ignite0.portables().metadata("TestObject3");
-
-        assertNotNull(meta);
-        assertEquals(2, meta.fields().size());
-
-        IgniteCache<Integer, PortableObject> cache = ignite0.cache(null).withKeepPortable();
-
-        PortableObject obj = cache.get(1);
-
-        assertEquals(Integer.valueOf(2), obj.field("f2"));
-        assertNull(obj.field("f1"));
-
-        meta = obj.metaData();
-
-        assertNotNull(meta);
-        assertEquals(2, meta.fields().size());
-
-        Collection<PortableMetadata> meta1 = ignite1.portables().metadata();
-        Collection<PortableMetadata> meta2 = ignite1.portables().metadata();
-
-        assertEquals(meta1.size(), meta2.size());
-
-        for (PortableMetadata m1 : meta1) {
-            boolean found = false;
-
-            for (PortableMetadata m2 : meta1) {
-                if (m1.typeName().equals(m2.typeName())) {
-                    assertEquals(m1.affinityKeyFieldName(), m2.affinityKeyFieldName());
-                    assertEquals(m1.fields(), m2.fields());
-
-                    found = true;
-
-                    break;
-                }
-            }
-
-            assertTrue(found);
-        }
-    }
-
-    /**
-     *
-     */
-    static class TestObject1 {
-        /** */
-        private int val1;
-
-        /** */
-        private int val2;
-
-        /**
-         * @param val1 Value 1.
-         * @param val2 Value 2.
-         */
-        public TestObject1(int val1, int val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TestObject1 that = (TestObject1)o;
-
-            return val1 == that.val1;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val1;
-        }
-    }
-
-    /**
-     *
-     */
-    static class TestObject2 {
-        /** */
-        private int val1;
-
-        /** */
-        private int val2;
-
-        /**
-         * @param val1 Value 1.
-         * @param val2 Value 2.
-         */
-        public TestObject2(int val1, int val2) {
-            this.val1 = val1;
-            this.val2 = val2;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            TestObject2 that = (TestObject2)o;
-
-            return val2 == that.val2;
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return val2;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractDataStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractDataStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractDataStreamerSelfTest.java
new file mode 100644
index 0000000..a51cbbf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractDataStreamerSelfTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jsr166.LongAdder8;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public abstract class GridCacheIgniteObjectsAbstractDataStreamerSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int THREAD_CNT = 64;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheMode(cacheMode());
+        cacheCfg.setAtomicityMode(atomicityMode());
+        cacheCfg.setNearConfiguration(nearConfiguration());
+        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new IgniteObjectConfiguration(TestObject.class.getName())));
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /**
+     * @return Sync mode.
+     */
+    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
+        return PRIMARY_SYNC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode cacheMode();
+
+    /**
+     * @return Atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
+    /**
+     * @return Near configuration.
+     */
+    protected abstract NearCacheConfiguration nearConfiguration();
+
+    /**
+     * @return Grid count.
+     */
+    protected int gridCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("BusyWait")
+    public void testGetPut() throws Exception {
+        final AtomicBoolean flag = new AtomicBoolean();
+
+        final LongAdder8 cnt = new LongAdder8();
+
+        try (IgniteDataStreamer<Object, Object> ldr = grid(0).dataStreamer(null)) {
+            IgniteInternalFuture<?> f = multithreadedAsync(
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        while (!flag.get()) {
+                            ldr.addData(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
+
+                            cnt.add(1);
+                        }
+
+                        return null;
+                    }
+                },
+                THREAD_CNT
+            );
+
+            for (int i = 0; i < 30 && !f.isDone(); i++)
+                Thread.sleep(1000);
+
+            flag.set(true);
+
+            f.get();
+        }
+
+        info("Operations in 30 sec: " + cnt.sum());
+    }
+
+    /**
+     */
+    private static class TestObject implements IgniteObjectMarshalAware, Serializable {
+        /** */
+        private int val;
+
+        /**
+         */
+        private TestObject() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        private TestObject(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            return obj instanceof TestObject && ((TestObject)obj).val == val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+            writer.writeInt("val", val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+            val = reader.readInt("val");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractMultiThreadedSelfTest.java
new file mode 100644
index 0000000..5620c3c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/portable/GridCacheIgniteObjectsAbstractMultiThreadedSelfTest.java
@@ -0,0 +1,231 @@
+/*
+ * 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.processors.cache.portable;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jsr166.LongAdder8;
+
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
+
+/**
+ * Test for portable objects stored in cache.
+ */
+public abstract class GridCacheIgniteObjectsAbstractMultiThreadedSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final int THREAD_CNT = 64;
+
+    /** */
+    private static final AtomicInteger idxGen = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cacheCfg = new CacheConfiguration();
+
+        cacheCfg.setCacheMode(cacheMode());
+        cacheCfg.setAtomicityMode(atomicityMode());
+        cacheCfg.setNearConfiguration(nearConfiguration());
+        cacheCfg.setWriteSynchronizationMode(writeSynchronizationMode());
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setTypeConfigurations(Arrays.asList(
+            new IgniteObjectConfiguration(TestObject.class.getName())));
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /**
+     * @return Sync mode.
+     */
+    protected CacheWriteSynchronizationMode writeSynchronizationMode() {
+        return PRIMARY_SYNC;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(gridCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return Cache mode.
+     */
+    protected abstract CacheMode cacheMode();
+
+    /**
+     * @return Atomicity mode.
+     */
+    protected abstract CacheAtomicityMode atomicityMode();
+
+    /**
+     * @return Distribution mode.
+     */
+    protected abstract NearCacheConfiguration nearConfiguration();
+
+    /**
+     * @return Grid count.
+     */
+    protected int gridCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("BusyWait") public void testGetPut() throws Exception {
+        final AtomicBoolean flag = new AtomicBoolean();
+
+        final LongAdder8 cnt = new LongAdder8();
+
+        IgniteInternalFuture<?> f = multithreadedAsync(
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    int threadId = idxGen.getAndIncrement() % 2;
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    while (!flag.get()) {
+                        IgniteCache<Object, Object> c = jcache(rnd.nextInt(gridCount()));
+
+                        switch (threadId) {
+                            case 0:
+                                // Put/get/remove portable -> portable.
+
+                                c.put(new TestObject(rnd.nextInt(10000)), new TestObject(rnd.nextInt(10000)));
+
+                                IgniteCache<Object, Object> p2 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
+
+                                IgniteObject v = (IgniteObject)p2.get(new TestObject(rnd.nextInt(10000)));
+
+                                if (v != null)
+                                    v.deserialize();
+
+                                c.remove(new TestObject(rnd.nextInt(10000)));
+
+                                break;
+
+                            case 1:
+                                // Put/get int -> portable.
+                                c.put(rnd.nextInt(10000), new TestObject(rnd.nextInt(10000)));
+
+                                IgniteCache<Integer, IgniteObject> p4 = ((IgniteCacheProxy<Object, Object>)c).keepPortable();
+
+                                IgniteObject v1 = p4.get(rnd.nextInt(10000));
+
+                                if (v1 != null)
+                                    v1.deserialize();
+
+                                p4.remove(rnd.nextInt(10000));
+
+                                break;
+
+                            default:
+                                assert false;
+                        }
+
+                        cnt.add(3);
+                    }
+
+                    return null;
+                }
+            },
+            THREAD_CNT
+        );
+
+        for (int i = 0; i < 30 && !f.isDone(); i++)
+            Thread.sleep(1000);
+
+        flag.set(true);
+
+        f.get();
+
+        info("Operations in 30 sec: " + cnt.sum());
+    }
+
+    /**
+     */
+    private static class TestObject implements IgniteObjectMarshalAware, Serializable {
+        /** */
+        private int val;
+
+        /**
+         */
+        private TestObject() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        private TestObject(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            return obj instanceof TestObject && ((TestObject)obj).val == val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
+            writer.writeInt("val", val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
+            val = reader.readInt("val");
+        }
+    }
+}
\ No newline at end of file


[15/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectWriterExImpl.java
new file mode 100644
index 0000000..de2b810
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/IgniteObjectWriterExImpl.java
@@ -0,0 +1,1892 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
+import org.apache.ignite.internal.portable.streams.PortableOutputStream;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectRawWriter;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.IOException;
+import java.io.ObjectOutput;
+import java.lang.reflect.InvocationTargetException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.IdentityHashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BOOLEAN_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.BYTE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CHAR_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLASS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.COL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DATE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DECIMAL_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.DOUBLE_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.ENUM_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.FLOAT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.INT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.LONG_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.MAP_ENTRY;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OBJ_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.OPTM_MARSH;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.PORTABLE_OBJ;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.RAW_DATA_OFF_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.SHORT_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.STRING_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TIMESTAMP_ARR;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.TOTAL_LEN_POS;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UNREGISTERED_TYPE_ID;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID;
+import static org.apache.ignite.internal.portable.GridPortableMarshaller.UUID_ARR;
+
+ /**
+ * Portable writer implementation.
+ */
+public class IgniteObjectWriterExImpl implements IgniteObjectWriter, IgniteObjectRawWriterEx, ObjectOutput {
+    /** Length: integer. */
+    private static final int LEN_INT = 4;
+
+    /** */
+    private static final int INIT_CAP = 1024;
+
+    /** */
+    private final PortableContext ctx;
+
+    /** */
+    private final WriterContext wCtx;
+
+    /** */
+    private final int start;
+
+    /** */
+    private int mark;
+
+    /** */
+    private Class<?> cls;
+
+    /** */
+    private int typeId;
+
+    /** */
+    private boolean allowFields = true;
+
+    /** */
+    private boolean metaEnabled;
+
+    /** */
+    private int metaHashSum;
+
+    /**
+     * @param ctx Context.
+     * @param off Start offset.
+     */
+    IgniteObjectWriterExImpl(PortableContext ctx, int off) {
+        this.ctx = ctx;
+
+        PortableOutputStream out = new PortableHeapOutputStream(off + INIT_CAP);
+
+        out.position(off);
+
+        wCtx = new WriterContext(out, null);
+
+        start = off;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param out Output stream.
+     * @param off Start offset.
+     */
+    IgniteObjectWriterExImpl(PortableContext ctx, PortableOutputStream out, int off) {
+        this.ctx = ctx;
+
+        wCtx = new WriterContext(out, null);
+
+        start = off;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param off Start offset.
+     * @param typeId Type ID.
+     */
+    public IgniteObjectWriterExImpl(PortableContext ctx, int off, int typeId, boolean metaEnabled) {
+        this(ctx, off);
+
+        this.typeId = typeId;
+
+        this.metaEnabled = metaEnabled;
+    }
+
+    /**
+     * @param ctx Context.
+     * @param wCtx Writer context.
+     */
+    private IgniteObjectWriterExImpl(PortableContext ctx, WriterContext wCtx) {
+        this.ctx = ctx;
+        this.wCtx = wCtx;
+
+        start = wCtx.out.position();
+    }
+
+    /**
+     * Close the writer releasing resources if necessary.
+     */
+    @Override public void close() {
+        wCtx.out.close();
+    }
+
+    /**
+     * @return Meta data hash sum or {@code null} if meta data is disabled.
+     */
+    @Nullable Integer metaDataHashSum() {
+        return metaEnabled ? metaHashSum : null;
+    }
+
+    /**
+     * @param obj Object.
+     * @param detached Detached or not.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void marshal(Object obj, boolean detached) throws IgniteObjectException {
+        assert obj != null;
+
+        cls = obj.getClass();
+
+        PortableClassDescriptor desc = ctx.descriptorForClass(cls);
+
+        if (desc == null)
+            throw new IgniteObjectException("Object is not portable: [class=" + cls + ']');
+
+        if (desc.excluded()) {
+            doWriteByte(NULL);
+            return;
+        }
+
+        if (desc.useOptimizedMarshaller()) {
+            writeByte(OPTM_MARSH);
+
+            try {
+                byte[] arr = ctx.optimizedMarsh().marshal(obj);
+
+                writeInt(arr.length);
+
+                write(arr);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteObjectException("Failed to marshal object with optimized marshaller: " + obj, e);
+            }
+
+            return;
+        }
+
+        if (desc.getWriteReplaceMethod() != null) {
+            Object replace;
+
+            try {
+                replace = desc.getWriteReplaceMethod().invoke(obj);
+            }
+            catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+            catch (InvocationTargetException e) {
+                if (e.getTargetException() instanceof IgniteObjectException)
+                    throw (IgniteObjectException)e.getTargetException();
+
+                throw new IgniteObjectException("Failed to execute writeReplace() method on " + obj, e);
+            }
+
+            if (replace == null) {
+                doWriteByte(NULL);
+                return;
+            }
+
+            if (cls != replace.getClass()) {
+                cls = replace.getClass();
+
+                desc = ctx.descriptorForClass(cls);
+
+                if (desc == null)
+                    throw new IgniteObjectException("Object is not portable: [class=" + cls + ']');
+            }
+
+            obj = replace;
+        }
+
+        typeId = desc.typeId();
+
+        metaEnabled = ctx.isMetaDataEnabled(typeId);
+
+        if (detached)
+            wCtx.resetHandles();
+
+        desc.write(obj, this);
+    }
+
+    /**
+     * @param obj Object.
+     * @return Handle.
+     */
+    int handle(Object obj) {
+        assert obj != null;
+
+        return wCtx.handle(obj);
+    }
+
+    /**
+     * @return Array.
+     */
+    public byte[] array() {
+        return wCtx.out.arrayCopy();
+    }
+
+    /**
+     * @return Output stream.
+     */
+    public PortableOutputStream outputStream() {
+        return wCtx.out;
+    }
+
+    /**
+     * @return Stream current position.
+     */
+    int position() {
+        return wCtx.out.position();
+    }
+
+    /**
+     * Sets new position.
+     *
+     * @param pos Position.
+     */
+    void position(int pos) {
+        wCtx.out.position(pos);
+    }
+
+     /**
+     * @param bytes Number of bytes to reserve.
+     * @return Offset.
+     */
+    public int reserve(int bytes) {
+        int pos = wCtx.out.position();
+
+        wCtx.out.position(pos + bytes);
+
+        return pos;
+    }
+
+    /**
+     * @param bytes Number of bytes to reserve.
+     * @return Offset.
+     */
+    public int reserveAndMark(int bytes) {
+        int off0 = reserve(bytes);
+
+        mark = wCtx.out.position();
+
+        return off0;
+    }
+
+    /**
+     * @param off Offset.
+     */
+    public void writeDelta(int off) {
+        wCtx.out.writeInt(off, wCtx.out.position() - mark);
+    }
+
+    /**
+     *
+     */
+    public void writeLength() {
+        wCtx.out.writeInt(start + TOTAL_LEN_POS, wCtx.out.position() - start);
+    }
+
+    /**
+     *
+     */
+    public void writeRawOffsetIfNeeded() {
+        if (allowFields)
+            wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
+    }
+
+    /**
+     * @param val Byte array.
+     */
+    public void write(byte[] val) {
+        assert val != null;
+
+        wCtx.out.writeByteArray(val);
+    }
+
+    /**
+     * @param val Byte array.
+     * @param off Offset.
+     * @param len Length.
+     */
+    public void write(byte[] val, int off, int len) {
+        assert val != null;
+
+        wCtx.out.write(val, off, len);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteByte(byte val) {
+        wCtx.out.writeByte(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteShort(short val) {
+        wCtx.out.writeShort(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteInt(int val) {
+        wCtx.out.writeInt(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteLong(long val) {
+        wCtx.out.writeLong(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteFloat(float val) {
+        wCtx.out.writeFloat(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteDouble(double val) {
+        wCtx.out.writeDouble(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteChar(char val) {
+        wCtx.out.writeChar(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    public void doWriteBoolean(boolean val) {
+        wCtx.out.writeBoolean(val);
+    }
+
+    /**
+     * @param val String value.
+     */
+    public void doWriteDecimal(@Nullable BigDecimal val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(DECIMAL);
+
+            BigInteger intVal = val.unscaledValue();
+
+            if (intVal.signum() == -1) {
+                intVal = intVal.negate();
+
+                wCtx.out.writeInt(val.scale() | 0x80000000);
+            }
+            else
+                wCtx.out.writeInt(val.scale());
+
+            byte[] vals = intVal.toByteArray();
+
+            wCtx.out.writeInt(vals.length);
+            wCtx.out.writeByteArray(vals);
+        }
+    }
+
+    /**
+     * @param val String value.
+     */
+    public void doWriteString(@Nullable String val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(STRING);
+
+            if (ctx.isConvertString()) {
+                doWriteBoolean(true);
+
+                byte[] strArr = val.getBytes(UTF_8);
+
+                doWriteInt(strArr.length);
+
+                wCtx.out.writeByteArray(strArr);
+            }
+            else {
+                doWriteBoolean(false);
+
+                char[] strArr = val.toCharArray();
+
+                doWriteInt(strArr.length);
+
+                wCtx.out.writeCharArray(strArr);
+            }
+        }
+    }
+
+    /**
+     * @param uuid UUID.
+     */
+    public void doWriteUuid(@Nullable UUID uuid) {
+        if (uuid == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(UUID);
+            doWriteLong(uuid.getMostSignificantBits());
+            doWriteLong(uuid.getLeastSignificantBits());
+        }
+    }
+
+    /**
+     * @param date Date.
+     */
+    public void doWriteDate(@Nullable Date date) {
+        if (date == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(DATE);
+            doWriteLong(date.getTime());
+        }
+    }
+
+     /**
+      * @param ts Timestamp.
+      */
+     public void doWriteTimestamp(@Nullable Timestamp ts) {
+         if (ts== null)
+             doWriteByte(NULL);
+         else {
+             doWriteByte(TIMESTAMP);
+             doWriteLong(ts.getTime());
+             doWriteInt(ts.getNanos() % 1000000);
+         }
+     }
+
+    /**
+     * @param obj Object.
+     * @param detached Detached or not.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    public void doWriteObject(@Nullable Object obj, boolean detached) throws IgniteObjectException {
+        if (obj == null)
+            doWriteByte(NULL);
+        else {
+            WriterContext wCtx = detached ? new WriterContext(this.wCtx.out, this.wCtx.handles) : this.wCtx;
+
+            IgniteObjectWriterExImpl writer = new IgniteObjectWriterExImpl(ctx, wCtx);
+
+            writer.marshal(obj, detached);
+
+            if (detached)
+                this.wCtx.out = wCtx.out;
+        }
+    }
+
+    /**
+     * @param val Byte array.
+     */
+    void doWriteByteArray(@Nullable byte[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(BYTE_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeByteArray(val);
+        }
+    }
+
+    /**
+     * @param val Short array.
+     */
+    void doWriteShortArray(@Nullable short[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(SHORT_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeShortArray(val);
+        }
+    }
+
+    /**
+     * @param val Integer array.
+     */
+    void doWriteIntArray(@Nullable int[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(INT_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeIntArray(val);
+        }
+    }
+
+    /**
+     * @param val Long array.
+     */
+    void doWriteLongArray(@Nullable long[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(LONG_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeLongArray(val);
+        }
+    }
+
+    /**
+     * @param val Float array.
+     */
+    void doWriteFloatArray(@Nullable float[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(FLOAT_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeFloatArray(val);
+        }
+    }
+
+    /**
+     * @param val Double array.
+     */
+    void doWriteDoubleArray(@Nullable double[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(DOUBLE_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeDoubleArray(val);
+        }
+    }
+
+    /**
+     * @param val Char array.
+     */
+    void doWriteCharArray(@Nullable char[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(CHAR_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeCharArray(val);
+        }
+    }
+
+    /**
+     * @param val Boolean array.
+     */
+    void doWriteBooleanArray(@Nullable boolean[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(BOOLEAN_ARR);
+            doWriteInt(val.length);
+
+            wCtx.out.writeBooleanArray(val);
+        }
+    }
+
+    /**
+     * @param val Array of strings.
+     */
+    void doWriteDecimalArray(@Nullable BigDecimal[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(DECIMAL_ARR);
+            doWriteInt(val.length);
+
+            for (BigDecimal str : val)
+                doWriteDecimal(str);
+        }
+    }
+
+    /**
+     * @param val Array of strings.
+     */
+    void doWriteStringArray(@Nullable String[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(STRING_ARR);
+            doWriteInt(val.length);
+
+            for (String str : val)
+                doWriteString(str);
+        }
+    }
+
+    /**
+     * @param val Array of UUIDs.
+     */
+    void doWriteUuidArray(@Nullable UUID[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(UUID_ARR);
+            doWriteInt(val.length);
+
+            for (UUID uuid : val)
+                doWriteUuid(uuid);
+        }
+    }
+
+    /**
+     * @param val Array of dates.
+     */
+    void doWriteDateArray(@Nullable Date[] val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            doWriteByte(DATE_ARR);
+            doWriteInt(val.length);
+
+            for (Date date : val)
+                doWriteDate(date);
+        }
+    }
+
+     /**
+      * @param val Array of timestamps.
+      */
+     void doWriteTimestampArray(@Nullable Timestamp[] val) {
+         if (val == null)
+             doWriteByte(NULL);
+         else {
+             if (tryWriteAsHandle(val))
+                 return;
+
+             doWriteByte(TIMESTAMP_ARR);
+             doWriteInt(val.length);
+
+             for (Timestamp ts : val)
+                 doWriteTimestamp(ts);
+         }
+     }
+
+    /**
+     * @param val Array of objects.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void doWriteObjectArray(@Nullable Object[] val) throws IgniteObjectException {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(val))
+                return;
+
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
+
+            doWriteByte(OBJ_ARR);
+
+            if (desc.registered())
+                doWriteInt(desc.typeId());
+            else {
+                doWriteInt(UNREGISTERED_TYPE_ID);
+                doWriteString(val.getClass().getComponentType().getName());
+            }
+
+            doWriteInt(val.length);
+
+            for (Object obj : val)
+                doWriteObject(obj, false);
+        }
+    }
+
+    /**
+     * @param col Collection.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void doWriteCollection(@Nullable Collection<?> col) throws IgniteObjectException {
+        if (col == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(col))
+                return;
+
+            doWriteByte(COL);
+            doWriteInt(col.size());
+            doWriteByte(ctx.collectionType(col.getClass()));
+
+            for (Object obj : col)
+                doWriteObject(obj, false);
+        }
+    }
+
+    /**
+     * @param map Map.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void doWriteMap(@Nullable Map<?, ?> map) throws IgniteObjectException {
+        if (map == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(map))
+                return;
+
+            doWriteByte(MAP);
+            doWriteInt(map.size());
+            doWriteByte(ctx.mapType(map.getClass()));
+
+            for (Map.Entry<?, ?> e : map.entrySet()) {
+                doWriteObject(e.getKey(), false);
+                doWriteObject(e.getValue(), false);
+            }
+        }
+    }
+
+    /**
+     * @param e Map entry.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void doWriteMapEntry(@Nullable Map.Entry<?, ?> e) throws IgniteObjectException {
+        if (e == null)
+            doWriteByte(NULL);
+        else {
+            if (tryWriteAsHandle(e))
+                return;
+
+            doWriteByte(MAP_ENTRY);
+            doWriteObject(e.getKey(), false);
+            doWriteObject(e.getValue(), false);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void doWriteEnum(@Nullable Enum<?> val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass());
+
+            doWriteByte(ENUM);
+
+            if (desc.registered())
+                doWriteInt(desc.typeId());
+            else {
+                doWriteInt(UNREGISTERED_TYPE_ID);
+                doWriteString(val.getClass().getName());
+            }
+
+            doWriteInt(val.ordinal());
+        }
+    }
+
+    /**
+     * @param val Array.
+     */
+    void doWriteEnumArray(@Nullable Object[] val) {
+        assert val == null || val.getClass().getComponentType().isEnum();
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            PortableClassDescriptor desc = ctx.descriptorForClass(val.getClass().getComponentType());
+            doWriteByte(ENUM_ARR);
+
+            if (desc.registered())
+                doWriteInt(desc.typeId());
+            else {
+                doWriteInt(UNREGISTERED_TYPE_ID);
+                doWriteString(val.getClass().getComponentType().getName());
+            }
+
+            doWriteInt(val.length);
+
+            // TODO: Denis: Redundant data for each element of the array.
+            for (Object o : val)
+                doWriteEnum((Enum<?>)o);
+        }
+    }
+
+    /**
+     * @param val Class.
+     */
+    void doWriteClass(@Nullable Class val) {
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            PortableClassDescriptor desc = ctx.descriptorForClass(val);
+
+            doWriteByte(CLASS);
+
+            if (desc.registered())
+                doWriteInt(desc.typeId());
+            else {
+                doWriteInt(UNREGISTERED_TYPE_ID);
+                doWriteString(val.getClass().getName());
+            }
+        }
+    }
+
+    /**
+     * @param po Portable object.
+     */
+    public void doWritePortableObject(@Nullable IgniteObjectImpl po) {
+        if (po == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(PORTABLE_OBJ);
+
+            byte[] poArr = po.array();
+
+            doWriteInt(poArr.length);
+
+            wCtx.out.writeByteArray(poArr);
+
+            doWriteInt(po.start());
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeByteField(@Nullable Byte val) {
+        doWriteInt(val != null ? 2 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(BYTE);
+            doWriteByte(val);
+        }
+    }
+
+    /**
+     * @param val Class.
+     */
+    void writeClassField(@Nullable Class val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteClass(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeShortField(@Nullable Short val) {
+        doWriteInt(val != null ? 3 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(SHORT);
+            doWriteShort(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeIntField(@Nullable Integer val) {
+        doWriteInt(val != null ? 5 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(INT);
+            doWriteInt(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeLongField(@Nullable Long val) {
+        doWriteInt(val != null ? 9 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(LONG);
+            doWriteLong(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeFloatField(@Nullable Float val) {
+        doWriteInt(val != null ? 5 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(FLOAT);
+            doWriteFloat(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDoubleField(@Nullable Double val) {
+        doWriteInt(val != null ? 9 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(DOUBLE);
+            doWriteDouble(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeCharField(@Nullable Character val) {
+        doWriteInt(val != null ? 3 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(CHAR);
+            doWriteChar(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeBooleanField(@Nullable Boolean val) {
+        doWriteInt(val != null ? 2 : 1);
+
+        if (val == null)
+            doWriteByte(NULL);
+        else {
+            doWriteByte(BOOLEAN);
+            doWriteBoolean(val);
+        }
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDecimalField(@Nullable BigDecimal val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteDecimal(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeStringField(@Nullable String val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteString(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeUuidField(@Nullable UUID val) {
+        doWriteInt(val != null ? 17 : 1);
+        doWriteUuid(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDateField(@Nullable Date val) {
+        doWriteInt(val != null ? 9 : 1);
+        doWriteDate(val);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeTimestampField(@Nullable Timestamp val) {
+        doWriteInt(val != null ? 13 : 1);
+        doWriteTimestamp(val);
+    }
+
+    /**
+     * @param obj Object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writeObjectField(@Nullable Object obj) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWriteObject(obj, false);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeByteArrayField(@Nullable byte[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteByteArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeShortArrayField(@Nullable short[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteShortArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeIntArrayField(@Nullable int[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteIntArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeLongArrayField(@Nullable long[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteLongArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeFloatArrayField(@Nullable float[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteFloatArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDoubleArrayField(@Nullable double[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteDoubleArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeCharArrayField(@Nullable char[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteCharArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeBooleanArrayField(@Nullable boolean[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteBooleanArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDecimalArrayField(@Nullable BigDecimal[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteDecimalArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeStringArrayField(@Nullable String[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteStringArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeUuidArrayField(@Nullable UUID[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteUuidArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeDateArrayField(@Nullable Date[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteDateArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeTimestampArrayField(@Nullable Timestamp[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteTimestampArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writeObjectArrayField(@Nullable Object[] val) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWriteObjectArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param col Collection.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writeCollectionField(@Nullable Collection<?> col) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWriteCollection(col);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param map Map.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writeMapField(@Nullable Map<?, ?> map) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWriteMap(map);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param e Map entry.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writeMapEntryField(@Nullable Map.Entry<?, ?> e) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWriteMapEntry(e);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeEnumField(@Nullable Enum<?> val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteEnum(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param val Value.
+     */
+    void writeEnumArrayField(@Nullable Object[] val) {
+        int lenPos = reserveAndMark(4);
+
+        doWriteEnumArray(val);
+
+        writeDelta(lenPos);
+    }
+
+    /**
+     * @param po Portable object.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
+     */
+    void writePortableObjectField(@Nullable IgniteObjectImpl po) throws IgniteObjectException {
+        int lenPos = reserveAndMark(4);
+
+        doWritePortableObject(po);
+
+        writeDelta(lenPos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws IgniteObjectException {
+        writeFieldId(fieldName, BYTE);
+        writeByteField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(byte val) throws IgniteObjectException {
+        doWriteByte(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws IgniteObjectException {
+        writeFieldId(fieldName, SHORT);
+        writeShortField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(short val) throws IgniteObjectException {
+        doWriteShort(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws IgniteObjectException {
+        writeFieldId(fieldName, INT);
+        writeIntField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(int val) throws IgniteObjectException {
+        doWriteInt(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws IgniteObjectException {
+        writeFieldId(fieldName, LONG);
+        writeLongField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(long val) throws IgniteObjectException {
+        doWriteLong(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws IgniteObjectException {
+        writeFieldId(fieldName, FLOAT);
+        writeFloatField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(float val) throws IgniteObjectException {
+        doWriteFloat(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws IgniteObjectException {
+        writeFieldId(fieldName, DOUBLE);
+        writeDoubleField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(double val) throws IgniteObjectException {
+        doWriteDouble(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws IgniteObjectException {
+        writeFieldId(fieldName, CHAR);
+        writeCharField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(char val) throws IgniteObjectException {
+        doWriteChar(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws IgniteObjectException {
+        writeFieldId(fieldName, BOOLEAN);
+        writeBooleanField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(boolean val) throws IgniteObjectException {
+        doWriteBoolean(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws IgniteObjectException {
+        writeFieldId(fieldName, DECIMAL);
+        writeDecimalField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimal(@Nullable BigDecimal val) throws IgniteObjectException {
+        doWriteDecimal(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws IgniteObjectException {
+        writeFieldId(fieldName, STRING);
+        writeStringField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(@Nullable String val) throws IgniteObjectException {
+        doWriteString(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuid(String fieldName, @Nullable UUID val) throws IgniteObjectException {
+        writeFieldId(fieldName, UUID);
+        writeUuidField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuid(@Nullable UUID val) throws IgniteObjectException {
+        doWriteUuid(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDate(String fieldName, @Nullable Date val) throws IgniteObjectException {
+        writeFieldId(fieldName, DATE);
+        writeDateField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDate(@Nullable Date val) throws IgniteObjectException {
+        doWriteDate(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws IgniteObjectException {
+        writeFieldId(fieldName, TIMESTAMP);
+        writeTimestampField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestamp(@Nullable Timestamp val) throws IgniteObjectException {
+        doWriteTimestamp(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws IgniteObjectException {
+        writeFieldId(fieldName, OBJ);
+        writeObjectField(obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(@Nullable Object obj) throws IgniteObjectException {
+        doWriteObject(obj, false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectDetached(@Nullable Object obj) throws IgniteObjectException {
+        doWriteObject(obj, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, BYTE_ARR);
+        writeByteArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(@Nullable byte[] val) throws IgniteObjectException {
+        doWriteByteArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, SHORT_ARR);
+        writeShortArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(@Nullable short[] val) throws IgniteObjectException {
+        doWriteShortArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, INT_ARR);
+        writeIntArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(@Nullable int[] val) throws IgniteObjectException {
+        doWriteIntArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, LONG_ARR);
+        writeLongArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(@Nullable long[] val) throws IgniteObjectException {
+        doWriteLongArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, FLOAT_ARR);
+        writeFloatArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(@Nullable float[] val) throws IgniteObjectException {
+        doWriteFloatArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, DOUBLE_ARR);
+        writeDoubleArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(@Nullable double[] val) throws IgniteObjectException {
+        doWriteDoubleArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, CHAR_ARR);
+        writeCharArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(@Nullable char[] val) throws IgniteObjectException {
+        doWriteCharArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, BOOLEAN_ARR);
+        writeBooleanArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(@Nullable boolean[] val) throws IgniteObjectException {
+        doWriteBooleanArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, DECIMAL_ARR);
+        writeDecimalArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDecimalArray(@Nullable BigDecimal[] val) throws IgniteObjectException {
+        doWriteDecimalArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, STRING_ARR);
+        writeStringArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(@Nullable String[] val) throws IgniteObjectException {
+        doWriteStringArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, UUID_ARR);
+        writeUuidArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUuidArray(@Nullable UUID[] val) throws IgniteObjectException {
+        doWriteUuidArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDateArray(String fieldName, @Nullable Date[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, DATE_ARR);
+        writeDateArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDateArray(@Nullable Date[] val) throws IgniteObjectException {
+        doWriteDateArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, TIMESTAMP_ARR);
+        writeTimestampArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeTimestampArray(@Nullable Timestamp[] val) throws IgniteObjectException {
+        doWriteTimestampArray(val);
+    }
+
+     /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, OBJ_ARR);
+        writeObjectArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(@Nullable Object[] val) throws IgniteObjectException {
+        doWriteObjectArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, COL);
+        writeCollectionField(col);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(@Nullable Collection<T> col) throws IgniteObjectException {
+        doWriteCollection(col);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map)
+        throws IgniteObjectException {
+        writeFieldId(fieldName, MAP);
+        writeMapField(map);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(@Nullable Map<K, V> map) throws IgniteObjectException {
+        doWriteMap(map);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IgniteObjectException {
+        writeFieldId(fieldName, ENUM);
+        writeEnumField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(T val) throws IgniteObjectException {
+        doWriteEnum(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IgniteObjectException {
+        writeFieldId(fieldName, ENUM_ARR);
+        writeEnumArrayField(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(T[] val) throws IgniteObjectException {
+        doWriteEnumArray(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteObjectRawWriter rawWriter() {
+        if (allowFields) {
+            wCtx.out.writeInt(start + RAW_DATA_OFF_POS, wCtx.out.position() - start);
+
+            allowFields = false;
+        }
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public PortableOutputStream out() {
+        return wCtx.out;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBytes(String s) throws IOException {
+        int len = s.length();
+
+        writeInt(len);
+
+        for (int i = 0; i < len; i++)
+            writeByte(s.charAt(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChars(String s) throws IOException {
+        int len = s.length();
+
+        writeInt(len);
+
+        for (int i = 0; i < len; i++)
+            writeChar(s.charAt(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUTF(String s) throws IOException {
+        writeString(s);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(int v) throws IOException {
+        doWriteByte((byte)v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(int v) throws IOException {
+        doWriteShort((short)v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(int v) throws IOException {
+        doWriteChar((char)v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(int b) throws IOException {
+        doWriteByte((byte)b);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush() throws IOException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public int reserveInt() {
+        return reserve(LEN_INT);
+    }
+
+     /** {@inheritDoc} */
+    @Override public void writeInt(int pos, int val) throws IgniteObjectException {
+        wCtx.out.writeInt(pos, val);
+    }
+
+    /**
+     * @param fieldName Field name.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If fields are not allowed.
+     */
+    private void writeFieldId(String fieldName, byte fieldType) throws IgniteObjectException {
+        A.notNull(fieldName, "fieldName");
+
+        if (!allowFields)
+            throw new IgniteObjectException("Individual field can't be written after raw writer is acquired " +
+                "via rawWriter() method. Consider fixing serialization logic for class: " + cls.getName());
+
+        int id = ctx.fieldId(typeId, fieldName);
+
+        if (metaEnabled)
+            metaHashSum = 31 * metaHashSum + (id + fieldType);
+
+        doWriteInt(id);
+    }
+
+     /**
+      * Attempts to write the object as a handle.
+      *
+      * @param obj Object to write.
+      * @return {@code true} if the object has been written as a handle.
+      */
+     boolean tryWriteAsHandle(Object obj) {
+         int handle = handle(obj);
+
+         if (handle >= 0) {
+             doWriteByte(GridPortableMarshaller.HANDLE);
+             doWriteInt(handle);
+
+             return true;
+         }
+
+         return false;
+     }
+
+    /**
+     * Create new writer with same context.
+     * @param typeId type
+     * @return New writer.
+     */
+    public IgniteObjectWriterExImpl newWriter(int typeId) {
+        IgniteObjectWriterExImpl res = new IgniteObjectWriterExImpl(ctx, wCtx);
+
+        res.typeId = typeId;
+
+        return res;
+    }
+
+    /**
+     * @return Portable context.
+     */
+    public PortableContext context() {
+        return ctx;
+    }
+
+    /** */
+    private static class WriterContext {
+        /** */
+        private Map<Object, Integer> handles = new IdentityHashMap<>();
+
+        /** Output stream. */
+        private PortableOutputStream out;
+
+        /**
+         * Constructor.
+         *
+         * @param out Output stream.
+         * @param handles Handles.
+         */
+        private WriterContext(PortableOutputStream out, Map<Object, Integer> handles) {
+            this.out = out;
+            this.handles = handles == null ? new IdentityHashMap<Object, Integer>() : handles;
+        }
+
+        /**
+         * @param obj Object.
+         * @return Handle.
+         */
+        private int handle(Object obj) {
+            assert obj != null;
+
+            Integer h = handles.get(obj);
+
+            if (h != null)
+                return out.position() - h;
+            else {
+                handles.put(obj, out.position());
+
+                return -1;
+            }
+        }
+
+        /**
+         *
+         */
+        private void resetHandles() {
+            handles = new IdentityHashMap<>();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index 0a9974e..2470837 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -41,10 +41,10 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableSerializer;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectSerializer;
 import org.jetbrains.annotations.Nullable;
 
 import static java.lang.reflect.Modifier.isStatic;
@@ -61,7 +61,7 @@ public class PortableClassDescriptor {
     private final Class<?> cls;
 
     /** */
-    private final PortableSerializer serializer;
+    private final IgniteObjectSerializer serializer;
 
     /** */
     private final Mode mode;
@@ -114,7 +114,7 @@ public class PortableClassDescriptor {
      * @param keepDeserialized Keep deserialized flag.
      * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
      * @param predefined Whether the class is predefined or not.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     PortableClassDescriptor(
         PortableContext ctx,
@@ -122,13 +122,13 @@ public class PortableClassDescriptor {
         boolean userType,
         int typeId,
         String typeName,
-        @Nullable PortableIdMapper idMapper,
-        @Nullable PortableSerializer serializer,
+        @Nullable IgniteObjectIdMapper idMapper,
+        @Nullable IgniteObjectSerializer serializer,
         boolean metaDataEnabled,
         boolean keepDeserialized,
         boolean registered,
         boolean predefined
-    ) throws PortableException {
+    ) throws IgniteObjectException {
         assert ctx != null;
         assert cls != null;
 
@@ -220,12 +220,12 @@ public class PortableClassDescriptor {
                             String name = f.getName();
 
                             if (!names.add(name))
-                                throw new PortableException("Duplicate field name: " + name);
+                                throw new IgniteObjectException("Duplicate field name: " + name);
 
                             int fieldId = idMapper.fieldId(typeId, name);
 
                             if (!ids.add(fieldId))
-                                throw new PortableException("Duplicate field ID: " + name);
+                                throw new IgniteObjectException("Duplicate field ID: " + name);
 
                             FieldInfo fieldInfo = new FieldInfo(f, fieldId);
 
@@ -241,7 +241,7 @@ public class PortableClassDescriptor {
 
             default:
                 // Should never happen.
-                throw new PortableException("Invalid mode: " + mode);
+                throw new IgniteObjectException("Invalid mode: " + mode);
         }
 
         if (mode == Mode.PORTABLE || mode == Mode.EXTERNALIZABLE || mode == Mode.OBJECT) {
@@ -323,9 +323,9 @@ public class PortableClassDescriptor {
     /**
      * @param obj Object.
      * @param writer Writer.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    void write(Object obj, PortableWriterExImpl writer) throws PortableException {
+    void write(Object obj, IgniteObjectWriterExImpl writer) throws IgniteObjectException {
         assert obj != null;
         assert writer != null;
 
@@ -504,7 +504,7 @@ public class PortableClassDescriptor {
                 break;
 
             case PORTABLE_OBJ:
-                writer.doWritePortableObject((PortableObjectImpl)obj);
+                writer.doWritePortableObject((IgniteObjectImpl)obj);
 
                 break;
 
@@ -513,19 +513,19 @@ public class PortableClassDescriptor {
                     if (serializer != null)
                         serializer.writePortable(obj, writer);
                     else
-                        ((PortableMarshalAware)obj).writePortable(writer);
+                        ((IgniteObjectMarshalAware)obj).writePortable(writer);
 
                     writer.writeRawOffsetIfNeeded();
                     writer.writeLength();
 
-                    if (obj.getClass() != PortableMetaDataImpl.class
+                    if (obj.getClass() != IgniteObjectMetaDataImpl.class
                         && ctx.isMetaDataChanged(typeId, writer.metaDataHashSum())) {
-                        PortableMetaDataCollector metaCollector = new PortableMetaDataCollector(typeName);
+                        IgniteObjectMetaDataCollector metaCollector = new IgniteObjectMetaDataCollector(typeName);
 
                         if (serializer != null)
                             serializer.writePortable(obj, metaCollector);
                         else
-                            ((PortableMarshalAware)obj).writePortable(metaCollector);
+                            ((IgniteObjectMarshalAware)obj).writePortable(metaCollector);
 
                         ctx.updateMetaData(typeId, typeName, metaCollector.meta());
                     }
@@ -539,7 +539,7 @@ public class PortableClassDescriptor {
                         ((Externalizable)obj).writeExternal(writer);
                     }
                     catch (IOException e) {
-                        throw new PortableException("Failed to write Externalizable object: " + obj, e);
+                        throw new IgniteObjectException("Failed to write Externalizable object: " + obj, e);
                     }
 
                     writer.writeLength();
@@ -566,9 +566,9 @@ public class PortableClassDescriptor {
     /**
      * @param reader Reader.
      * @return Object.
-     * @throws PortableException If failed.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
      */
-    Object read(PortableReaderExImpl reader) throws PortableException {
+    Object read(IgniteObjectReaderExImpl reader) throws IgniteObjectException {
         assert reader != null;
 
         Object res;
@@ -582,7 +582,7 @@ public class PortableClassDescriptor {
                 if (serializer != null)
                     serializer.readPortable(res, reader);
                 else
-                    ((PortableMarshalAware)res).readPortable(reader);
+                    ((IgniteObjectMarshalAware)res).readPortable(reader);
 
                 break;
 
@@ -595,7 +595,7 @@ public class PortableClassDescriptor {
                     ((Externalizable)res).readExternal(reader);
                 }
                 catch (IOException | ClassNotFoundException e) {
-                    throw new PortableException("Failed to read Externalizable object: " +
+                    throw new IgniteObjectException("Failed to read Externalizable object: " +
                         res.getClass().getName(), e);
                 }
 
@@ -627,10 +627,10 @@ public class PortableClassDescriptor {
                 throw new RuntimeException(e);
             }
             catch (InvocationTargetException e) {
-                if (e.getTargetException() instanceof PortableException)
-                    throw (PortableException)e.getTargetException();
+                if (e.getTargetException() instanceof IgniteObjectException)
+                    throw (IgniteObjectException)e.getTargetException();
 
-                throw new PortableException("Failed to execute readResolve() method on " + res, e);
+                throw new IgniteObjectException("Failed to execute readResolve() method on " + res, e);
             }
         }
 
@@ -642,7 +642,7 @@ public class PortableClassDescriptor {
      * @param writer Writer.
      * @return Whether further write is needed.
      */
-    private boolean writeHeader(Object obj, PortableWriterExImpl writer) {
+    private boolean writeHeader(Object obj, IgniteObjectWriterExImpl writer) {
         if (writer.tryWriteAsHandle(obj))
             return false;
 
@@ -674,25 +674,25 @@ public class PortableClassDescriptor {
 
     /**
      * @return Instance.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    private Object newInstance() throws PortableException {
+    private Object newInstance() throws IgniteObjectException {
         assert ctor != null;
 
         try {
             return ctor.newInstance();
         }
         catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
-            throw new PortableException("Failed to instantiate instance: " + cls, e);
+            throw new IgniteObjectException("Failed to instantiate instance: " + cls, e);
         }
     }
 
     /**
      * @param cls Class.
      * @return Constructor.
-     * @throws PortableException If constructor doesn't exist.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException If constructor doesn't exist.
      */
-    @Nullable private static Constructor<?> constructor(Class<?> cls) throws PortableException {
+    @Nullable private static Constructor<?> constructor(Class<?> cls) throws IgniteObjectException {
         assert cls != null;
 
         try {
@@ -703,7 +703,7 @@ public class PortableClassDescriptor {
             return ctor;
         }
         catch (IgniteCheckedException e) {
-            throw new PortableException("Failed to get constructor for class: " + cls.getName(), e);
+            throw new IgniteObjectException("Failed to get constructor for class: " + cls.getName(), e);
         }
     }
 
@@ -795,9 +795,9 @@ public class PortableClassDescriptor {
             return Mode.TIMESTAMP_ARR;
         else if (cls.isArray())
             return cls.getComponentType().isEnum() ? Mode.ENUM_ARR : Mode.OBJ_ARR;
-        else if (cls == PortableObjectImpl.class)
+        else if (cls == IgniteObjectImpl.class)
             return Mode.PORTABLE_OBJ;
-        else if (PortableMarshalAware.class.isAssignableFrom(cls))
+        else if (IgniteObjectMarshalAware.class.isAssignableFrom(cls))
             return Mode.PORTABLE;
         else if (Externalizable.class.isAssignableFrom(cls))
             return Mode.EXTERNALIZABLE;
@@ -807,7 +807,7 @@ public class PortableClassDescriptor {
             return Mode.COL;
         else if (Map.class.isAssignableFrom(cls))
             return Mode.MAP;
-        else if (cls == PortableObjectImpl.class)
+        else if (cls == IgniteObjectImpl.class)
             return Mode.PORTABLE_OBJ;
         else if (cls.isEnum())
             return Mode.ENUM;
@@ -853,9 +853,9 @@ public class PortableClassDescriptor {
         /**
          * @param obj Object.
          * @param writer Writer.
-         * @throws PortableException In case of error.
+         * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
          */
-        public void write(Object obj, PortableWriterExImpl writer) throws PortableException {
+        public void write(Object obj, IgniteObjectWriterExImpl writer) throws IgniteObjectException {
             assert obj != null;
             assert writer != null;
 
@@ -867,7 +867,7 @@ public class PortableClassDescriptor {
                 val = field.get(obj);
             }
             catch (IllegalAccessException e) {
-                throw new PortableException("Failed to get value for field: " + field, e);
+                throw new IgniteObjectException("Failed to get value for field: " + field, e);
             }
 
             switch (mode) {
@@ -1022,7 +1022,7 @@ public class PortableClassDescriptor {
                     break;
 
                 case PORTABLE_OBJ:
-                    writer.writePortableObjectField((PortableObjectImpl)val);
+                    writer.writePortableObjectField((IgniteObjectImpl)val);
 
                     break;
 
@@ -1056,9 +1056,9 @@ public class PortableClassDescriptor {
         /**
          * @param obj Object.
          * @param reader Reader.
-         * @throws PortableException In case of error.
+         * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
          */
-        public void read(Object obj, PortableReaderExImpl reader) throws PortableException {
+        public void read(Object obj, IgniteObjectReaderExImpl reader) throws IgniteObjectException {
             Object val = null;
 
             switch (mode) {
@@ -1248,7 +1248,7 @@ public class PortableClassDescriptor {
                     field.set(obj, val);
             }
             catch (IllegalAccessException e) {
-                throw new PortableException("Failed to set value for field: " + field, e);
+                throw new IgniteObjectException("Failed to set value for field: " + field, e);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 9c11ca8..2822441 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -50,6 +50,8 @@ import java.util.concurrent.ConcurrentSkipListSet;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
@@ -61,12 +63,12 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectInvalidClassException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectSerializer;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -78,10 +80,10 @@ public class PortableContext implements Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** */
-    static final PortableIdMapper DFLT_ID_MAPPER = new IdMapperWrapper(null);
+    static final IgniteObjectIdMapper DFLT_ID_MAPPER = new IdMapperWrapper(null);
 
     /** */
-    static final PortableIdMapper BASIC_CLS_ID_MAPPER = new BasicClassIdMapper();
+    static final IgniteObjectIdMapper BASIC_CLS_ID_MAPPER = new BasicClassIdMapper();
 
     /** */
     static final char[] LOWER_CASE_CHARS;
@@ -121,10 +123,10 @@ public class PortableContext implements Externalizable {
     private final Map<Class<? extends Map>, Byte> mapTypes = new HashMap<>();
 
     /** */
-    private final Map<Integer, PortableIdMapper> mappers = new ConcurrentHashMap8<>(0);
+    private final Map<Integer, IgniteObjectIdMapper> mappers = new ConcurrentHashMap8<>(0);
 
     /** */
-    private final Map<String, PortableIdMapper> typeMappers = new ConcurrentHashMap8<>(0);
+    private final Map<String, IgniteObjectIdMapper> typeMappers = new ConcurrentHashMap8<>(0);
 
     /** */
     private Map<Integer, Boolean> metaEnabled = new HashMap<>(0);
@@ -139,6 +141,9 @@ public class PortableContext implements Externalizable {
     private String gridName;
 
     /** */
+    private IgniteConfiguration igniteCfg;
+
+    /** */
     private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
 
     /** */
@@ -159,13 +164,14 @@ public class PortableContext implements Externalizable {
 
     /**
      * @param metaHnd Meta data handler.
-     * @param gridName Grid name.
+     * @param igniteCfg Ignite configuration.
      */
-    public PortableContext(PortableMetaDataHandler metaHnd, @Nullable String gridName) {
+    public PortableContext(PortableMetaDataHandler metaHnd, @Nullable IgniteConfiguration igniteCfg) {
         assert metaHnd != null;
 
         this.metaHnd = metaHnd;
-        this.gridName = gridName;
+        this.igniteCfg = igniteCfg;
+        gridName = igniteCfg.getGridName();
 
         colTypes.put(ArrayList.class, GridPortableMarshaller.ARR_LIST);
         colTypes.put(LinkedList.class, GridPortableMarshaller.LINKED_LIST);
@@ -232,15 +238,15 @@ public class PortableContext implements Externalizable {
 
         // IDs range [200..1000] is used by Ignite internal APIs.
 
-        registerPredefinedType(PortableObjectImpl.class, 200);
-        registerPredefinedType(PortableMetaDataImpl.class, 201);
+        registerPredefinedType(IgniteObjectImpl.class, 200);
+        registerPredefinedType(IgniteObjectMetaDataImpl.class, 201);
     }
 
     /**
      * @param marsh Portable marshaller.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public void configure(PortableMarshaller marsh) throws PortableException {
+    public void configure(PortableMarshaller marsh) throws IgniteObjectException {
         if (marsh == null)
             return;
 
@@ -271,20 +277,20 @@ public class PortableContext implements Externalizable {
      * @param globalKeepDeserialized Keep deserialized flag.
      * @param clsNames Class names.
      * @param typeCfgs Type configurations.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     private void configure(
-        PortableIdMapper globalIdMapper,
-        PortableSerializer globalSerializer,
+        IgniteObjectIdMapper globalIdMapper,
+        IgniteObjectSerializer globalSerializer,
         boolean globalMetaDataEnabled,
         boolean globalKeepDeserialized,
         Collection<String> clsNames,
-        Collection<PortableTypeConfiguration> typeCfgs
-    ) throws PortableException {
+        Collection<IgniteObjectConfiguration> typeCfgs
+    ) throws IgniteObjectException {
         TypeDescriptors descs = new TypeDescriptors();
 
         if (clsNames != null) {
-            PortableIdMapper idMapper = new IdMapperWrapper(globalIdMapper);
+            IgniteObjectIdMapper idMapper = new IdMapperWrapper(globalIdMapper);
 
             for (String clsName : clsNames) {
                 if (clsName.endsWith(".*")) { // Package wildcard
@@ -300,21 +306,26 @@ public class PortableContext implements Externalizable {
             }
         }
 
+        Map<String, String> affFields = new HashMap<>();
+
+        for (CacheKeyConfiguration keyCfg : igniteCfg.getCacheKeyConfiguration())
+            affFields.put(keyCfg.getTypeName(), keyCfg.getAffinityKeyFieldName());
+
         if (typeCfgs != null) {
-            for (PortableTypeConfiguration typeCfg : typeCfgs) {
+            for (IgniteObjectConfiguration typeCfg : typeCfgs) {
                 String clsName = typeCfg.getClassName();
 
                 if (clsName == null)
-                    throw new PortableException("Class name is required for portable type configuration.");
+                    throw new IgniteObjectException("Class name is required for portable type configuration.");
 
-                PortableIdMapper idMapper = globalIdMapper;
+                IgniteObjectIdMapper idMapper = globalIdMapper;
 
                 if (typeCfg.getIdMapper() != null)
                     idMapper = typeCfg.getIdMapper();
 
                 idMapper = new IdMapperWrapper(idMapper);
 
-                PortableSerializer serializer = globalSerializer;
+                IgniteObjectSerializer serializer = globalSerializer;
 
                 if (typeCfg.getSerializer() != null)
                     serializer = typeCfg.getSerializer();
@@ -328,11 +339,11 @@ public class PortableContext implements Externalizable {
                     String pkgName = clsName.substring(0, clsName.length() - 2);
 
                     for (String clsName0 : classesInPackage(pkgName))
-                        descs.add(clsName0, idMapper, serializer, typeCfg.getAffinityKeyFieldName(),
+                        descs.add(clsName0, idMapper, serializer, affFields.get(clsName0),
                             metaDataEnabled, keepDeserialized, true);
                 }
                 else
-                    descs.add(clsName, idMapper, serializer, typeCfg.getAffinityKeyFieldName(),
+                    descs.add(clsName, idMapper, serializer, affFields.get(clsName),
                         metaDataEnabled, keepDeserialized, false);
             }
         }
@@ -414,10 +425,10 @@ public class PortableContext implements Externalizable {
     /**
      * @param cls Class.
      * @return Class descriptor.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     public PortableClassDescriptor descriptorForClass(Class<?> cls)
-        throws PortableException {
+        throws IgniteObjectException {
         assert cls != null;
 
         PortableClassDescriptor desc = descByCls.get(cls);
@@ -461,10 +472,10 @@ public class PortableContext implements Externalizable {
             desc = descByCls.get(cls);
         }
         catch (ClassNotFoundException e) {
-            throw new PortableInvalidClassException(e);
+            throw new IgniteObjectInvalidClassException(e);
         }
         catch (IgniteCheckedException e) {
-            throw new PortableException("Failed resolve class for ID: " + typeId, e);
+            throw new IgniteObjectException("Failed resolve class for ID: " + typeId, e);
         }
 
         if (desc == null) {
@@ -523,7 +534,7 @@ public class PortableContext implements Externalizable {
 
         String typeName = typeName(cls.getName());
 
-        PortableIdMapper idMapper = idMapper(typeName);
+        IgniteObjectIdMapper idMapper = idMapper(typeName);
 
         int typeId = idMapper.typeId(typeName);
 
@@ -531,7 +542,7 @@ public class PortableContext implements Externalizable {
             registered = marshCtx.registerClass(typeId, cls);
         }
         catch (IgniteCheckedException e) {
-            throw new PortableException("Failed to register class.", e);
+            throw new IgniteObjectException("Failed to register class.", e);
         }
 
         PortableClassDescriptor desc = new PortableClassDescriptor(this,
@@ -616,8 +627,8 @@ public class PortableContext implements Externalizable {
      * @param typeId Type ID.
      * @return Instance of ID mapper.
      */
-    public PortableIdMapper idMapper(int typeId) {
-        PortableIdMapper idMapper = mappers.get(typeId);
+    public IgniteObjectIdMapper idMapper(int typeId) {
+        IgniteObjectIdMapper idMapper = mappers.get(typeId);
 
         if (idMapper != null)
             return idMapper;
@@ -632,15 +643,15 @@ public class PortableContext implements Externalizable {
      * @param typeName Type name.
      * @return Instance of ID mapper.
      */
-    private PortableIdMapper idMapper(String typeName) {
-        PortableIdMapper idMapper = typeMappers.get(typeName);
+    private IgniteObjectIdMapper idMapper(String typeName) {
+        IgniteObjectIdMapper idMapper = typeMappers.get(typeName);
 
         return idMapper != null ? idMapper : DFLT_ID_MAPPER;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeString(out, gridName);
+        U.writeString(out, igniteCfg.getGridName());
     }
 
     /** {@inheritDoc} */
@@ -703,16 +714,16 @@ public class PortableContext implements Externalizable {
      * @param affKeyFieldName Affinity key field name.
      * @param metaDataEnabled Metadata enabled flag.
      * @param keepDeserialized Keep deserialized flag.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
     @SuppressWarnings("ErrorNotRethrown")
     public void registerUserType(String clsName,
-        PortableIdMapper idMapper,
-        @Nullable PortableSerializer serializer,
+        IgniteObjectIdMapper idMapper,
+        @Nullable IgniteObjectSerializer serializer,
         @Nullable String affKeyFieldName,
         boolean metaDataEnabled,
         boolean keepDeserialized)
-        throws PortableException {
+        throws IgniteObjectException {
         assert idMapper != null;
 
         Class<?> cls = null;
@@ -728,10 +739,10 @@ public class PortableContext implements Externalizable {
 
         //Workaround for IGNITE-1358
         if (predefinedTypes.get(id) != null)
-            throw new PortableException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+            throw new IgniteObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
         if (mappers.put(id, idMapper) != null)
-            throw new PortableException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+            throw new IgniteObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
 
         String typeName = typeName(clsName);
 
@@ -762,15 +773,15 @@ public class PortableContext implements Externalizable {
             descByCls.put(cls, desc);
         }
 
-        metaHnd.addMeta(id, new PortableMetaDataImpl(typeName, fieldsMeta, affKeyFieldName));
+        metaHnd.addMeta(id, new IgniteObjectMetaDataImpl(typeName, fieldsMeta, affKeyFieldName));
     }
 
     /**
      * @param typeId Type ID.
      * @return Meta data.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    @Nullable public PortableMetadata metaData(int typeId) throws PortableException {
+    @Nullable public IgniteObjectMetadata metaData(int typeId) throws IgniteObjectException {
         return metaHnd != null ? metaHnd.metadata(typeId) : null;
     }
 
@@ -809,18 +820,18 @@ public class PortableContext implements Externalizable {
      * @param typeId Type ID.
      * @param typeName Type name.
      * @param fields Fields map.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws PortableException {
-        updateMetaData(typeId, new PortableMetaDataImpl(typeName, fields, null));
+    public void updateMetaData(int typeId, String typeName, Map<String, String> fields) throws IgniteObjectException {
+        updateMetaData(typeId, new IgniteObjectMetaDataImpl(typeName, fields, null));
     }
 
     /**
      * @param typeId Type ID.
      * @param meta Meta data.
-     * @throws PortableException In case of error.
+     * @throws org.apache.ignite.igniteobject.IgniteObjectException In case of error.
      */
-    public void updateMetaData(int typeId, PortableMetaDataImpl meta) throws PortableException {
+    public void updateMetaData(int typeId, IgniteObjectMetaDataImpl meta) throws IgniteObjectException {
         metaHnd.addMeta(typeId, meta);
     }
 
@@ -894,14 +905,14 @@ public class PortableContext implements Externalizable {
 
     /**
      */
-    private static class IdMapperWrapper implements PortableIdMapper {
+    private static class IdMapperWrapper implements IgniteObjectIdMapper {
         /** */
-        private final PortableIdMapper mapper;
+        private final IgniteObjectIdMapper mapper;
 
         /**
          * @param mapper Custom ID mapper.
          */
-        private IdMapperWrapper(@Nullable PortableIdMapper mapper) {
+        private IdMapperWrapper(@Nullable IgniteObjectIdMapper mapper) {
             this.mapper = mapper;
         }
 
@@ -929,7 +940,7 @@ public class PortableContext implements Externalizable {
     /**
      * Basic class ID mapper.
      */
-    private static class BasicClassIdMapper implements PortableIdMapper {
+    private static class BasicClassIdMapper implements IgniteObjectIdMapper {
         /** {@inheritDoc} */
         @Override public int typeId(String clsName) {
             return clsName.hashCode();
@@ -958,16 +969,16 @@ public class PortableContext implements Externalizable {
          * @param metadataEnabled Metadata enabled flag.
          * @param keepDeserialized Keep deserialized flag.
          * @param canOverride Whether this descriptor can be override.
-         * @throws PortableException If failed.
+         * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
          */
         private void add(String clsName,
-            PortableIdMapper idMapper,
-            PortableSerializer serializer,
+            IgniteObjectIdMapper idMapper,
+            IgniteObjectSerializer serializer,
             String affKeyFieldName,
             boolean metadataEnabled,
             boolean keepDeserialized,
             boolean canOverride)
-            throws PortableException {
+            throws IgniteObjectException {
             TypeDescriptor desc = new TypeDescriptor(clsName,
                 idMapper,
                 serializer,
@@ -1002,10 +1013,10 @@ public class PortableContext implements Externalizable {
         private final String clsName;
 
         /** ID mapper. */
-        private PortableIdMapper idMapper;
+        private IgniteObjectIdMapper idMapper;
 
         /** Serializer. */
-        private PortableSerializer serializer;
+        private IgniteObjectSerializer serializer;
 
         /** Affinity key field name. */
         private String affKeyFieldName;
@@ -1030,7 +1041,7 @@ public class PortableContext implements Externalizable {
          * @param keepDeserialized Keep deserialized flag.
          * @param canOverride Whether this descriptor can be override.
          */
-        private TypeDescriptor(String clsName, PortableIdMapper idMapper, PortableSerializer serializer,
+        private TypeDescriptor(String clsName, IgniteObjectIdMapper idMapper, IgniteObjectSerializer serializer,
             String affKeyFieldName, boolean metadataEnabled, boolean keepDeserialized,
             boolean canOverride) {
             this.clsName = clsName;
@@ -1046,9 +1057,9 @@ public class PortableContext implements Externalizable {
          * Override portable class descriptor.
          *
          * @param other Other descriptor.
-         * @throws PortableException If failed.
+         * @throws org.apache.ignite.igniteobject.IgniteObjectException If failed.
          */
-        private void override(TypeDescriptor other) throws PortableException {
+        private void override(TypeDescriptor other) throws IgniteObjectException {
             assert clsName.equals(other.clsName);
 
             if (canOverride) {
@@ -1060,7 +1071,7 @@ public class PortableContext implements Externalizable {
                 canOverride = other.canOverride;
             }
             else if (!other.canOverride)
-                throw new PortableException("Duplicate explicit class definition in configuration: " + clsName);
+                throw new IgniteObjectException("Duplicate explicit class definition in configuration: " + clsName);
         }
     }
 


[06/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
deleted file mode 100644
index 331d9dc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderSelfTest.java
+++ /dev/null
@@ -1,1069 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.util.GridUnsafe;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import sun.misc.Unsafe;
-
-/**
- * Portable builder test.
- */
-public class GridPortableBuilderSelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
-            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
-
-        PortableTypeConfiguration customIdMapper = new PortableTypeConfiguration();
-
-        customIdMapper.setClassName(CustomIdMapper.class.getName());
-        customIdMapper.setIdMapper(new PortableIdMapper() {
-            @Override public int typeId(String clsName) {
-                return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
-            }
-
-            @Override public int fieldId(int typeId, String fieldName) {
-                return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
-            }
-        });
-
-        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
-
-        marsh.setConvertStringToBytes(useUtf8());
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @return Whether to use UTF8 strings.
-     */
-    protected boolean useUtf8() {
-        return true;
-    }
-
-    /**
-     *
-     */
-    public void testAllFieldsSerialization() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
-
-        GridTestUtils.deepEquals(obj, deserialized);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteField", (byte)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortField", (short)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((short)1, po.<Short>field("shortField").shortValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Integer>field("intField").intValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longField", 1L);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1L, po.<Long>field("longField").longValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatField", 1.0f);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleField", 1.0d);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charField", (char)1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals((char)1, po.<Character>field("charField").charValue());
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanField", true);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(po.<Boolean>field("booleanField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalField", BigDecimal.TEN);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringField", "str");
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals("str", po.<String>field("stringField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateField() throws Exception {
-        Date date = new Date();
-
-        assertEquals(date, builder("C").setField("d", date).build().<Date>field("d"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestampField() throws Exception {
-        Timestamp ts = new Timestamp(new Date().getTime());
-        ts.setNanos(1000);
-
-        assertEquals(ts, builder("C").setField("t", ts).build().<Timestamp>field("t"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID uuid = UUID.randomUUID();
-
-        builder.setField("uuidField", uuid);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(uuid, po.<UUID>field("uuidField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testByteArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShortArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("shortArrayField", new short[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testIntArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("intArrayField", new int[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testLongArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("longArrayField", new long[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testFloatArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("floatArrayField", new float[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDoubleArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("doubleArrayField", new double[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCharArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("charArrayField", new char[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBooleanArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("booleanArrayField", new boolean[] {true, false});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        boolean[] arr = po.field("booleanArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertTrue(arr[0]);
-        assertFalse(arr[1]);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDecimalArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testStringArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDateArrayField() throws Exception {
-        Date date1 = new Date();
-        Date date2 = new Date(date1.getTime() + 1000);
-
-        Date[] dateArr = new Date[] { date1, date2 };
-
-        assertTrue(Arrays.equals(dateArr, builder("C").setField("da", dateArr).build().<Date[]>field("da")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTimestampArrayField() throws Exception {
-        Timestamp ts1 = new Timestamp(new Date().getTime());
-        Timestamp ts2 = new Timestamp(new Date().getTime() + 1000);
-
-        ts1.setNanos(1000);
-        ts2.setNanos(2000);
-
-        Timestamp[] tsArr = new Timestamp[] { ts1, ts2 };
-
-        assertTrue(Arrays.equals(tsArr, builder("C").setField("ta", tsArr).build().<Timestamp[]>field("ta")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testUuidArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
-
-        builder.setField("uuidArrayField", arr);
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectField", new Value(1));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<PortableObject>field("objectField").<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testObjectArrayField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Object[] arr = po.field("objectArrayField");
-
-        assertEquals(2, arr.length);
-
-        assertEquals(1, ((PortableObject)arr[0]).<Value>deserialize().i);
-        assertEquals(2, ((PortableObject)arr[1]).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testCollectionField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        List<PortableObject> list = po.field("collectionField");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMapField() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        Map<PortableObject, PortableObject> map = po.field("mapField");
-
-        assertEquals(2, map.size());
-
-        for (Map.Entry<PortableObject, PortableObject> e : map.entrySet())
-            assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSeveralFields() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        assertEquals("class".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(111, po.<Integer>field("i").intValue());
-        assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
-        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
-        assertEquals(1, po.<PortableObject>field("obj").<Key>deserialize().i);
-
-        List<PortableObject> list = po.field("col");
-
-        assertEquals(2, list.size());
-
-        assertEquals(1, list.get(0).<Value>deserialize().i);
-        assertEquals(2, list.get(1).<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testOffheapPortable() throws Exception {
-        PortableBuilder builder = builder("Class");
-
-        builder.hashCode(100);
-
-        builder.setField("i", 111);
-        builder.setField("f", 111.111f);
-        builder.setField("iArr", new int[] {1, 2, 3});
-        builder.setField("obj", new Key(1));
-        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
-
-        PortableObject po = builder.build();
-
-        byte[] arr = ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
-
-        long ptr = UNSAFE.allocateMemory(arr.length + 5);
-
-        try {
-            long ptr0 = ptr;
-
-            UNSAFE.putBoolean(null, ptr0++, false);
-
-            UNSAFE.putInt(ptr0, arr.length);
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
-
-            PortableObject offheapObj = (PortableObject)
-                ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
-
-            assertEquals(PortableObjectOffheapImpl.class, offheapObj.getClass());
-
-            assertEquals("class".hashCode(), offheapObj.typeId());
-            assertEquals(100, offheapObj.hashCode());
-
-            assertEquals(111, offheapObj.<Integer>field("i").intValue());
-            assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
-            assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
-            assertEquals(1, offheapObj.<PortableObject>field("obj").<Key>deserialize().i);
-
-            List<PortableObject> list = offheapObj.field("col");
-
-            assertEquals(2, list.size());
-
-            assertEquals(1, list.get(0).<Value>deserialize().i);
-            assertEquals(2, list.get(1).<Value>deserialize().i);
-
-            assertEquals(po, offheapObj);
-            assertEquals(offheapObj, po);
-        }
-        finally {
-            UNSAFE.freeMemory(ptr);
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testBuildAndDeserialize() throws Exception {
-        PortableBuilder builder = builder(Value.class.getName());
-
-        builder.hashCode(100);
-
-        builder.setField("i", 1);
-
-        PortableObject po = builder.build();
-
-        assertEquals("value".hashCode(), po.typeId());
-        assertEquals(100, po.hashCode());
-
-        assertEquals(1, po.<Value>deserialize().i);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData2() throws Exception {
-        PortableBuilder builder = builder("org.test.MetaTest2");
-
-        builder.setField("objectField", "a", Object.class);
-
-        PortableObject po = builder.build();
-
-        PortableMetadata meta = po.metaData();
-
-        assertEquals("MetaTest2", meta.typeName());
-        assertEquals("Object", meta.fieldTypeName("objectField"));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetaData() throws Exception {
-        PortableBuilder builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 1);
-        builder.setField("byteArrayField", new byte[] {1, 2, 3});
-
-        PortableObject po = builder.build();
-
-        PortableMetadata meta = po.metaData();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        Collection<String> fields = meta.fields();
-
-        assertEquals(2, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-
-        builder = builder("org.test.MetaTest");
-
-        builder.hashCode(100);
-
-        builder.setField("intField", 2);
-        builder.setField("uuidField", UUID.randomUUID());
-
-        po = builder.build();
-
-        meta = po.metaData();
-
-        assertEquals("MetaTest", meta.typeName());
-
-        fields = meta.fields();
-
-        assertEquals(3, fields.size());
-
-        assertTrue(fields.contains("intField"));
-        assertTrue(fields.contains("byteArrayField"));
-        assertTrue(fields.contains("uuidField"));
-
-        assertEquals("int", meta.fieldTypeName("intField"));
-        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
-        assertEquals("UUID", meta.fieldTypeName("uuidField"));
-    }
-
-    /**
-     *
-     */
-    public void testGetFromCopiedObj() {
-        PortableObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
-
-        PortableBuilderImpl builder = builder(objStr);
-        assertEquals("aaa", builder.getField("str"));
-
-        builder.setField("str", "bbb");
-        assertEquals("bbb", builder.getField("str"));
-
-        assertNull(builder.getField("i_"));
-        assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCopyFromInnerObjects() {
-        ArrayList<Object> list = new ArrayList<>();
-        list.add(new TestObjectAllTypes());
-        list.add(list.get(0));
-
-        TestObjectContainer c = new TestObjectContainer(list);
-
-        PortableBuilderImpl builder = builder(toPortable(c));
-        builder.<List>getField("foo").add("!!!");
-
-        PortableObject res = builder.build();
-
-        TestObjectContainer deserialized = res.deserialize();
-
-        List deserializedList = (List)deserialized.foo;
-
-        assertSame(deserializedList.get(0), deserializedList.get(1));
-        assertEquals("!!!", deserializedList.get(2));
-        assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testSetPortableObject() {
-        PortableObject portableObj = builder(TestObjectContainer.class.getName())
-            .setField("foo", toPortable(new TestObjectAllTypes()))
-            .build();
-
-        assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
-    }
-
-    /**
-     *
-     */
-    public void testPlainPortableObjectCopyFrom() {
-        TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
-
-        PortableBuilderImpl builder = builder(toPortable(obj));
-        assertTrue(builder.getField("plainPortable") instanceof PortableObject);
-
-        TestObjectPlainPortable deserialized = builder.build().deserialize();
-        assertTrue(deserialized.plainPortable instanceof PortableObject);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        PortableBuilder builder = builder(TestObjectAllTypes.class.getName());
-
-        builder.setField("str", "a");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        PortableBuilder builder = builder(toPortable(obj));
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObjectAfterGet() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-        obj.enumArr = null;
-
-        PortableBuilderImpl builder = builder(toPortable(obj));
-
-        builder.getField("i_");
-
-        builder.removeField("str");
-
-        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-        outer.inner.outer = outer;
-        outer.foo = "a";
-
-        PortableBuilder builder = builder(toPortable(outer));
-
-        builder.setField("foo", "b");
-
-        TestObjectOuter res = builder.build().deserialize();
-
-        assertEquals("b", res.foo);
-        assertSame(res, res.inner.outer);
-    }
-
-    /**
-     * @return Portables.
-     */
-    private IgnitePortables portables() {
-        return grid(0).portables();
-    }
-
-    /**
-     * @param obj Object.
-     * @return Portable object.
-     */
-    private PortableObject toPortable(Object obj) {
-        return portables().toPortable(obj);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilder builder(int typeId) {
-        return portables().builder(typeId);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilder builder(String clsName) {
-        return portables().builder(clsName);
-    }
-
-    /**
-     * @return Builder.
-     */
-    private <T> PortableBuilderImpl builder(PortableObject obj) {
-        return (PortableBuilderImpl)portables().builder(obj);
-    }
-
-    /**
-     *
-     */
-    private static class CustomIdMapper {
-        /** */
-        private String str = "a";
-
-        /** */
-        private int i = 10;
-    }
-
-    /**
-     */
-    private static class Key {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Key() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Key(int i) {
-            this.i = i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object o) {
-            if (this == o)
-                return true;
-
-            if (o == null || getClass() != o.getClass())
-                return false;
-
-            Key key = (Key)o;
-
-            return i == key.i;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return i;
-        }
-    }
-
-    /**
-     */
-    private static class Value {
-        /** */
-        private int i;
-
-        /**
-         */
-        private Value() {
-            // No-op.
-        }
-
-        /**
-         * @param i Index.
-         */
-        private Value(int i) {
-            this.i = i;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java
deleted file mode 100644
index 2fce1a5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsAdditionalSelfTest.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-/**
- *
- */
-public class GridPortableBuilderStringAsCharsAdditionalSelfTest extends GridPortableBuilderAdditionalSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean useUtf8() {
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java
deleted file mode 100644
index 5c53233..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderStringAsCharsSelfTest.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-/**
- * Portable builder test.
- */
-public class GridPortableBuilderStringAsCharsSelfTest extends GridPortableBuilderSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean useUtf8() {
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
index bd9612c..d6e5a26 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableMarshallerCtxDisabledSelfTest.java
@@ -26,11 +26,11 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableReader;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMarshalAware;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectReader;
+import org.apache.ignite.igniteobject.IgniteObjectWriter;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -39,11 +39,11 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstractTest {
     /** */
     protected static final PortableMetaDataHandler META_HND = new PortableMetaDataHandler() {
-        @Override public void addMeta(int typeId, PortableMetadata meta) {
+        @Override public void addMeta(int typeId, IgniteObjectMetadata meta) {
             // No-op.
         }
 
-        @Override public PortableMetadata metadata(int typeId) {
+        @Override public IgniteObjectMetadata metadata(int typeId) {
             return null;
         }
     };
@@ -77,7 +77,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
 
         assertEquals(simpleObj, marsh.unmarshal(marsh.marshal(simpleObj), null));
 
-        SimplePortable simplePortable = new SimplePortable();
+        SimpleIgniteObject simplePortable = new SimpleIgniteObject();
 
         simplePortable.str = "portable";
         simplePortable.arr = new long[] {100, 200, 300};
@@ -179,7 +179,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
     /**
      *
      */
-    private static class SimplePortable implements PortableMarshalAware {
+    private static class SimpleIgniteObject implements IgniteObjectMarshalAware {
         /** */
         private String str;
 
@@ -187,13 +187,13 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
         private long[] arr;
 
         /** {@inheritDoc} */
-        @Override public void writePortable(PortableWriter writer) throws PortableException {
+        @Override public void writePortable(IgniteObjectWriter writer) throws IgniteObjectException {
             writer.writeString("str", str);
             writer.writeLongArray("longArr", arr);
         }
 
         /** {@inheritDoc} */
-        @Override public void readPortable(PortableReader reader) throws PortableException {
+        @Override public void readPortable(IgniteObjectReader reader) throws IgniteObjectException {
             str = reader.readString("str");
             arr = reader.readLongArray("longArr");
         }
@@ -206,7 +206,7 @@ public class GridPortableMarshallerCtxDisabledSelfTest extends GridCommonAbstrac
             if (o == null || getClass() != o.getClass())
                 return false;
 
-            SimplePortable that = (SimplePortable)o;
+            SimpleIgniteObject that = (SimpleIgniteObject)o;
 
             if (str != null ? !str.equals(that.str) : that.str != null)
                 return false;


[07/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderSelfTest.java
new file mode 100644
index 0000000..c6540e8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderSelfTest.java
@@ -0,0 +1,1069 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.portable;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteObjects;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.portable.builder.IgniteObjectBuilderImpl;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
+import org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectPlainPortable;
+import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.igniteobject.IgniteObjectBuilder;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import sun.misc.Unsafe;
+
+/**
+ * Portable builder test.
+ */
+public class GridIgniteObjectBuilderSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        PortableMarshaller marsh = new PortableMarshaller();
+
+        marsh.setClassNames(Arrays.asList(Key.class.getName(), Value.class.getName(),
+            "org.gridgain.grid.internal.util.portable.mutabletest.*"));
+
+        IgniteObjectConfiguration customIdMapper = new IgniteObjectConfiguration();
+
+        customIdMapper.setClassName(CustomIdMapper.class.getName());
+        customIdMapper.setIdMapper(new IgniteObjectIdMapper() {
+            @Override public int typeId(String clsName) {
+                return ~PortableContext.DFLT_ID_MAPPER.typeId(clsName);
+            }
+
+            @Override public int fieldId(int typeId, String fieldName) {
+                return typeId + ~PortableContext.DFLT_ID_MAPPER.fieldId(typeId, fieldName);
+            }
+        });
+
+        marsh.setTypeConfigurations(Collections.singleton(customIdMapper));
+
+        marsh.setConvertStringToBytes(useUtf8());
+
+        cfg.setMarshaller(marsh);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return Whether to use UTF8 strings.
+     */
+    protected boolean useUtf8() {
+        return true;
+    }
+
+    /**
+     *
+     */
+    public void testAllFieldsSerialization() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        TestObjectAllTypes deserialized = builder(toPortable(obj)).build().deserialize();
+
+        GridTestUtils.deepEquals(obj, deserialized);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByteField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("byteField", (byte)1);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((byte) 1, po.<Byte>field("byteField").byteValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("shortField", (short)1);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((short)1, po.<Short>field("shortField").shortValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 1);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<Integer>field("intField").intValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("longField", 1L);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1L, po.<Long>field("longField").longValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("floatField", 1.0f);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1.0f, po.<Float>field("floatField").floatValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("doubleField", 1.0d);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1.0d, po.<Double>field("doubleField").doubleValue(), 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("charField", (char)1);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals((char)1, po.<Character>field("charField").charValue());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBooleanField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("booleanField", true);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(po.<Boolean>field("booleanField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimalField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("decimalField", BigDecimal.TEN);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(BigDecimal.TEN, po.<String>field("decimalField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("stringField", "str");
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals("str", po.<String>field("stringField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateField() throws Exception {
+        Date date = new Date();
+
+        assertEquals(date, builder("C").setField("d", date).build().<Date>field("d"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestampField() throws Exception {
+        Timestamp ts = new Timestamp(new Date().getTime());
+        ts.setNanos(1000);
+
+        assertEquals(ts, builder("C").setField("t", ts).build().<Timestamp>field("t"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        UUID uuid = UUID.randomUUID();
+
+        builder.setField("uuidField", uuid);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(uuid, po.<UUID>field("uuidField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByteArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("byteArrayField", new byte[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new byte[] {1, 2, 3}, po.<byte[]>field("byteArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShortArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("shortArrayField", new short[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new short[] {1, 2, 3}, po.<short[]>field("shortArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIntArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("intArrayField", new int[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("intArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLongArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("longArrayField", new long[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new long[] {1, 2, 3}, po.<long[]>field("longArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloatArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("floatArrayField", new float[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new float[] {1, 2, 3}, po.<float[]>field("floatArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDoubleArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("doubleArrayField", new double[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new double[] {1, 2, 3}, po.<double[]>field("doubleArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCharArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("charArrayField", new char[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new char[] {1, 2, 3}, po.<char[]>field("charArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBooleanArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("booleanArrayField", new boolean[] {true, false});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        boolean[] arr = po.field("booleanArrayField");
+
+        assertEquals(2, arr.length);
+
+        assertTrue(arr[0]);
+        assertFalse(arr[1]);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDecimalArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("decimalArrayField", new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new BigDecimal[] {BigDecimal.ONE, BigDecimal.TEN}, po.<String[]>field("decimalArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStringArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("stringArrayField", new String[] {"str1", "str2", "str3"});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(new String[] {"str1", "str2", "str3"}, po.<String[]>field("stringArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDateArrayField() throws Exception {
+        Date date1 = new Date();
+        Date date2 = new Date(date1.getTime() + 1000);
+
+        Date[] dateArr = new Date[] { date1, date2 };
+
+        assertTrue(Arrays.equals(dateArr, builder("C").setField("da", dateArr).build().<Date[]>field("da")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestampArrayField() throws Exception {
+        Timestamp ts1 = new Timestamp(new Date().getTime());
+        Timestamp ts2 = new Timestamp(new Date().getTime() + 1000);
+
+        ts1.setNanos(1000);
+        ts2.setNanos(2000);
+
+        Timestamp[] tsArr = new Timestamp[] { ts1, ts2 };
+
+        assertTrue(Arrays.equals(tsArr, builder("C").setField("ta", tsArr).build().<Timestamp[]>field("ta")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUuidArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        UUID[] arr = new UUID[] {UUID.randomUUID(), UUID.randomUUID()};
+
+        builder.setField("uuidArrayField", arr);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertTrue(Arrays.equals(arr, po.<UUID[]>field("uuidArrayField")));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("objectField", new Value(1));
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<IgniteObject>field("objectField").<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObjectArrayField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("objectArrayField", new Value[] {new Value(1), new Value(2)});
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        Object[] arr = po.field("objectArrayField");
+
+        assertEquals(2, arr.length);
+
+        assertEquals(1, ((IgniteObject)arr[0]).<Value>deserialize().i);
+        assertEquals(2, ((IgniteObject)arr[1]).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCollectionField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("collectionField", Arrays.asList(new Value(1), new Value(2)));
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        List<IgniteObject> list = po.field("collectionField");
+
+        assertEquals(2, list.size());
+
+        assertEquals(1, list.get(0).<Value>deserialize().i);
+        assertEquals(2, list.get(1).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapField() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("mapField", F.asMap(new Key(1), new Value(1), new Key(2), new Value(2)));
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        Map<IgniteObject, IgniteObject> map = po.field("mapField");
+
+        assertEquals(2, map.size());
+
+        for (Map.Entry<IgniteObject, IgniteObject> e : map.entrySet())
+            assertEquals(e.getKey().<Key>deserialize().i, e.getValue().<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSeveralFields() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("i", 111);
+        builder.setField("f", 111.111f);
+        builder.setField("iArr", new int[] {1, 2, 3});
+        builder.setField("obj", new Key(1));
+        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+
+        IgniteObject po = builder.build();
+
+        assertEquals("class".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(111, po.<Integer>field("i").intValue());
+        assertEquals(111.111f, po.<Float>field("f").floatValue(), 0);
+        assertTrue(Arrays.equals(new int[] {1, 2, 3}, po.<int[]>field("iArr")));
+        assertEquals(1, po.<IgniteObject>field("obj").<Key>deserialize().i);
+
+        List<IgniteObject> list = po.field("col");
+
+        assertEquals(2, list.size());
+
+        assertEquals(1, list.get(0).<Value>deserialize().i);
+        assertEquals(2, list.get(1).<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOffheapPortable() throws Exception {
+        IgniteObjectBuilder builder = builder("Class");
+
+        builder.hashCode(100);
+
+        builder.setField("i", 111);
+        builder.setField("f", 111.111f);
+        builder.setField("iArr", new int[] {1, 2, 3});
+        builder.setField("obj", new Key(1));
+        builder.setField("col", Arrays.asList(new Value(1), new Value(2)));
+
+        IgniteObject po = builder.build();
+
+        byte[] arr = ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).marshal(po);
+
+        long ptr = UNSAFE.allocateMemory(arr.length + 5);
+
+        try {
+            long ptr0 = ptr;
+
+            UNSAFE.putBoolean(null, ptr0++, false);
+
+            UNSAFE.putInt(ptr0, arr.length);
+
+            UNSAFE.copyMemory(arr, BYTE_ARR_OFF, null, ptr0 + 4, arr.length);
+
+            IgniteObject offheapObj = (IgniteObject)
+                ((CacheObjectPortableProcessorImpl)(grid(0)).context().cacheObjects()).unmarshal(ptr, false);
+
+            assertEquals(IgniteObjectOffheapImpl.class, offheapObj.getClass());
+
+            assertEquals("class".hashCode(), offheapObj.typeId());
+            assertEquals(100, offheapObj.hashCode());
+
+            assertEquals(111, offheapObj.<Integer>field("i").intValue());
+            assertEquals(111.111f, offheapObj.<Float>field("f").floatValue(), 0);
+            assertTrue(Arrays.equals(new int[] {1, 2, 3}, offheapObj.<int[]>field("iArr")));
+            assertEquals(1, offheapObj.<IgniteObject>field("obj").<Key>deserialize().i);
+
+            List<IgniteObject> list = offheapObj.field("col");
+
+            assertEquals(2, list.size());
+
+            assertEquals(1, list.get(0).<Value>deserialize().i);
+            assertEquals(2, list.get(1).<Value>deserialize().i);
+
+            assertEquals(po, offheapObj);
+            assertEquals(offheapObj, po);
+        }
+        finally {
+            UNSAFE.freeMemory(ptr);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBuildAndDeserialize() throws Exception {
+        IgniteObjectBuilder builder = builder(Value.class.getName());
+
+        builder.hashCode(100);
+
+        builder.setField("i", 1);
+
+        IgniteObject po = builder.build();
+
+        assertEquals("value".hashCode(), po.typeId());
+        assertEquals(100, po.hashCode());
+
+        assertEquals(1, po.<Value>deserialize().i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetaData2() throws Exception {
+        IgniteObjectBuilder builder = builder("org.test.MetaTest2");
+
+        builder.setField("objectField", "a", Object.class);
+
+        IgniteObject po = builder.build();
+
+        IgniteObjectMetadata meta = po.metaData();
+
+        assertEquals("MetaTest2", meta.typeName());
+        assertEquals("Object", meta.fieldTypeName("objectField"));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetaData() throws Exception {
+        IgniteObjectBuilder builder = builder("org.test.MetaTest");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 1);
+        builder.setField("byteArrayField", new byte[] {1, 2, 3});
+
+        IgniteObject po = builder.build();
+
+        IgniteObjectMetadata meta = po.metaData();
+
+        assertEquals("MetaTest", meta.typeName());
+
+        Collection<String> fields = meta.fields();
+
+        assertEquals(2, fields.size());
+
+        assertTrue(fields.contains("intField"));
+        assertTrue(fields.contains("byteArrayField"));
+
+        assertEquals("int", meta.fieldTypeName("intField"));
+        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+
+        builder = builder("org.test.MetaTest");
+
+        builder.hashCode(100);
+
+        builder.setField("intField", 2);
+        builder.setField("uuidField", UUID.randomUUID());
+
+        po = builder.build();
+
+        meta = po.metaData();
+
+        assertEquals("MetaTest", meta.typeName());
+
+        fields = meta.fields();
+
+        assertEquals(3, fields.size());
+
+        assertTrue(fields.contains("intField"));
+        assertTrue(fields.contains("byteArrayField"));
+        assertTrue(fields.contains("uuidField"));
+
+        assertEquals("int", meta.fieldTypeName("intField"));
+        assertEquals("byte[]", meta.fieldTypeName("byteArrayField"));
+        assertEquals("UUID", meta.fieldTypeName("uuidField"));
+    }
+
+    /**
+     *
+     */
+    public void testGetFromCopiedObj() {
+        IgniteObject objStr = builder(TestObjectAllTypes.class.getName()).setField("str", "aaa").build();
+
+        IgniteObjectBuilderImpl builder = builder(objStr);
+        assertEquals("aaa", builder.getField("str"));
+
+        builder.setField("str", "bbb");
+        assertEquals("bbb", builder.getField("str"));
+
+        assertNull(builder.getField("i_"));
+        assertEquals("bbb", builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testCopyFromInnerObjects() {
+        ArrayList<Object> list = new ArrayList<>();
+        list.add(new TestObjectAllTypes());
+        list.add(list.get(0));
+
+        TestObjectContainer c = new TestObjectContainer(list);
+
+        IgniteObjectBuilderImpl builder = builder(toPortable(c));
+        builder.<List>getField("foo").add("!!!");
+
+        IgniteObject res = builder.build();
+
+        TestObjectContainer deserialized = res.deserialize();
+
+        List deserializedList = (List)deserialized.foo;
+
+        assertSame(deserializedList.get(0), deserializedList.get(1));
+        assertEquals("!!!", deserializedList.get(2));
+        assertTrue(deserializedList.get(0) instanceof TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testSetPortableObject() {
+        IgniteObject portableObj = builder(TestObjectContainer.class.getName())
+            .setField("foo", toPortable(new TestObjectAllTypes()))
+            .build();
+
+        assertTrue(portableObj.<TestObjectContainer>deserialize().foo instanceof TestObjectAllTypes);
+    }
+
+    /**
+     *
+     */
+    public void testPlainPortableObjectCopyFrom() {
+        TestObjectPlainPortable obj = new TestObjectPlainPortable(toPortable(new TestObjectAllTypes()));
+
+        IgniteObjectBuilderImpl builder = builder(toPortable(obj));
+        assertTrue(builder.getField("plainPortable") instanceof IgniteObject);
+
+        TestObjectPlainPortable deserialized = builder.build().deserialize();
+        assertTrue(deserialized.plainPortable instanceof IgniteObject);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromNewObject() {
+        IgniteObjectBuilder builder = builder(TestObjectAllTypes.class.getName());
+
+        builder.setField("str", "a");
+
+        builder.removeField("str");
+
+        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObject() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        IgniteObjectBuilder builder = builder(toPortable(obj));
+
+        builder.removeField("str");
+
+        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     *
+     */
+    public void testRemoveFromExistingObjectAfterGet() {
+        TestObjectAllTypes obj = new TestObjectAllTypes();
+        obj.setDefaultData();
+        obj.enumArr = null;
+
+        IgniteObjectBuilderImpl builder = builder(toPortable(obj));
+
+        builder.getField("i_");
+
+        builder.removeField("str");
+
+        assertNull(builder.build().<TestObjectAllTypes>deserialize().str);
+    }
+
+    /**
+     * @throws IgniteCheckedException If any error occurs.
+     */
+    public void testDontBrokeCyclicDependency() throws IgniteCheckedException {
+        TestObjectOuter outer = new TestObjectOuter();
+        outer.inner = new TestObjectInner();
+        outer.inner.outer = outer;
+        outer.foo = "a";
+
+        IgniteObjectBuilder builder = builder(toPortable(outer));
+
+        builder.setField("foo", "b");
+
+        TestObjectOuter res = builder.build().deserialize();
+
+        assertEquals("b", res.foo);
+        assertSame(res, res.inner.outer);
+    }
+
+    /**
+     * @return Portables.
+     */
+    private IgniteObjects portables() {
+        return grid(0).portables();
+    }
+
+    /**
+     * @param obj Object.
+     * @return Portable object.
+     */
+    private IgniteObject toPortable(Object obj) {
+        return portables().toPortable(obj);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private <T> IgniteObjectBuilder builder(int typeId) {
+        return portables().builder(typeId);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private <T> IgniteObjectBuilder builder(String clsName) {
+        return portables().builder(clsName);
+    }
+
+    /**
+     * @return Builder.
+     */
+    private <T> IgniteObjectBuilderImpl builder(IgniteObject obj) {
+        return (IgniteObjectBuilderImpl)portables().builder(obj);
+    }
+
+    /**
+     *
+     */
+    private static class CustomIdMapper {
+        /** */
+        private String str = "a";
+
+        /** */
+        private int i = 10;
+    }
+
+    /**
+     */
+    private static class Key {
+        /** */
+        private int i;
+
+        /**
+         */
+        private Key() {
+            // No-op.
+        }
+
+        /**
+         * @param i Index.
+         */
+        private Key(int i) {
+            this.i = i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            Key key = (Key)o;
+
+            return i == key.i;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return i;
+        }
+    }
+
+    /**
+     */
+    private static class Value {
+        /** */
+        private int i;
+
+        /**
+         */
+        private Value() {
+            // No-op.
+        }
+
+        /**
+         * @param i Index.
+         */
+        private Value(int i) {
+            this.i = i;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.java
new file mode 100644
index 0000000..49f8a39
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest.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;
+
+/**
+ *
+ */
+public class GridIgniteObjectBuilderStringAsCharsAdditionalSelfTest extends GridIgniteObjectBuilderAdditionalSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean useUtf8() {
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsSelfTest.java
new file mode 100644
index 0000000..e138e03
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridIgniteObjectBuilderStringAsCharsSelfTest.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;
+
+/**
+ * Portable builder test.
+ */
+public class GridIgniteObjectBuilderStringAsCharsSelfTest extends GridIgniteObjectBuilderSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean useUtf8() {
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
index 59084db..e2e5f29 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableAffinityKeySelfTest.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -32,7 +33,7 @@ import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProces
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -58,15 +59,18 @@ public class GridPortableAffinityKeySelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        PortableTypeConfiguration typeCfg = new PortableTypeConfiguration();
+        IgniteObjectConfiguration typeCfg = new IgniteObjectConfiguration();
 
         typeCfg.setClassName(TestObject.class.getName());
-        typeCfg.setAffinityKeyFieldName("affKey");
 
         PortableMarshaller marsh = new PortableMarshaller();
 
         marsh.setTypeConfigurations(Collections.singleton(typeCfg));
 
+        CacheKeyConfiguration keyCfg = new CacheKeyConfiguration(TestObject.class.getName(), "affKey");
+
+        cfg.setCacheKeyCfg(keyCfg);
+
         cfg.setMarshaller(marsh);
 
         if (!gridName.equals(getTestGridName(GRID_CNT))) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
deleted file mode 100644
index d8eff6d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/portable/GridPortableBuilderAdditionalSelfTest.java
+++ /dev/null
@@ -1,1289 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import java.lang.reflect.Field;
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgnitePortables;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.portable.builder.PortableBuilderEnum;
-import org.apache.ignite.internal.portable.builder.PortableBuilderImpl;
-import org.apache.ignite.internal.portable.mutabletest.GridPortableMarshalerAwareTestClass;
-import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
-import org.apache.ignite.internal.processors.cache.portable.IgnitePortablesImpl;
-import org.apache.ignite.internal.util.lang.GridMapEntry;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-
-import static org.apache.ignite.cache.CacheMode.REPLICATED;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Address;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.AddressBook;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.Company;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectAllTypes;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectArrayList;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectContainer;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectEnum;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectInner;
-import static org.apache.ignite.internal.portable.mutabletest.GridPortableTestClasses.TestObjectOuter;
-
-/**
- *
- */
-public class GridPortableBuilderAdditionalSelfTest extends GridCommonAbstractTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration cacheCfg = new CacheConfiguration();
-
-        cacheCfg.setCacheMode(REPLICATED);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        PortableMarshaller marsh = new PortableMarshaller();
-
-        marsh.setClassNames(Arrays.asList("org.apache.ignite.internal.portable.mutabletest.*"));
-
-        marsh.setConvertStringToBytes(useUtf8());
-
-        cfg.setMarshaller(marsh);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startGrids(1);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        jcache(0).clear();
-    }
-
-    /**
-     * @return Whether to use UTF8 strings.
-     */
-    protected boolean useUtf8() {
-        return true;
-    }
-
-    /**
-     * @return Portables API.
-     */
-    protected IgnitePortables portables() {
-        return grid(0).portables();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleTypeFieldRead() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilder mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            Object expVal = field.get(exp);
-            Object actVal = mutPo.getField(field.getName());
-
-            switch (field.getName()) {
-                case "anEnum":
-                    assertEquals(((PortableBuilderEnum)actVal).getOrdinal(), ((Enum)expVal).ordinal());
-                    break;
-
-                case "enumArr": {
-                    PortableBuilderEnum[] actArr = (PortableBuilderEnum[])actVal;
-                    Enum[] expArr = (Enum[])expVal;
-
-                    assertEquals(expArr.length, actArr.length);
-
-                    for (int i = 0; i < actArr.length; i++)
-                        assertEquals(expArr[i].ordinal(), actArr[i].getOrdinal());
-
-                    break;
-                }
-
-                case "entry":
-                    assertEquals(((Map.Entry)expVal).getKey(), ((Map.Entry)actVal).getKey());
-                    assertEquals(((Map.Entry)expVal).getValue(), ((Map.Entry)actVal).getValue());
-                    break;
-
-                default:
-                    assertTrue(field.getName(), Objects.deepEquals(expVal, actVal));
-                    break;
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    public void testSimpleTypeFieldSerialize() {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(exp);
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldOverride() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(new TestObjectAllTypes());
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields())
-            mutPo.setField(field.getName(), field.get(exp));
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        GridTestUtils.deepEquals(exp, res);
-    }
-
-    /**
-     * @throws Exception If any error occurs.
-     */
-    public void testSimpleTypeFieldSetNull() throws Exception {
-        TestObjectAllTypes exp = new TestObjectAllTypes();
-
-        exp.setDefaultData();
-
-        PortableBuilderImpl mutPo = wrap(exp);
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                mutPo.setField(field.getName(), null);
-        }
-
-        TestObjectAllTypes res = mutPo.build().deserialize();
-
-        for (Field field : TestObjectAllTypes.class.getDeclaredFields()) {
-            if (!field.getType().isPrimitive())
-                assertNull(field.getName(), field.get(res));
-        }
-    }
-
-    /**
-     * @throws IgniteCheckedException If any error occurs.
-     */
-    public void testMakeCyclicDependency() throws IgniteCheckedException {
-        TestObjectOuter outer = new TestObjectOuter();
-        outer.inner = new TestObjectInner();
-
-        PortableBuilderImpl mutOuter = wrap(outer);
-
-        PortableBuilderImpl mutInner = mutOuter.getField("inner");
-
-        mutInner.setField("outer", mutOuter);
-        mutInner.setField("foo", mutInner);
-
-        TestObjectOuter res = mutOuter.build().deserialize();
-
-        assertEquals(res, res.inner.outer);
-        assertEquals(res.inner, res.inner.foo);
-    }
-
-    /**
-     *
-     */
-    public void testDateArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.dateArr =  new Date[] {new Date(11111), new Date(11111), new Date(11111)};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Date[] arr = mutObj.getField("dateArr");
-        arr[0] = new Date(22222);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Date[] {new Date(22222), new Date(11111), new Date(11111)}, res.dateArr);
-    }
-
-    /**
-     *
-     */
-    public void testTimestampArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.tsArr = new Timestamp[] {new Timestamp(111222333), new Timestamp(222333444)};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Timestamp[] arr = mutObj.getField("tsArr");
-        arr[0] = new Timestamp(333444555);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Timestamp[] {new Timestamp(333444555), new Timestamp(222333444)}, res.tsArr);
-    }
-
-    /**
-     *
-     */
-    public void testUUIDArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.uuidArr = new UUID[] {new UUID(1, 1), new UUID(1, 1), new UUID(1, 1)};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        UUID[] arr = mutObj.getField("uuidArr");
-        arr[0] = new UUID(2, 2);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new UUID[] {new UUID(2, 2), new UUID(1, 1), new UUID(1, 1)}, res.uuidArr);
-    }
-
-    /**
-     *
-     */
-    public void testDecimalArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.bdArr = new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        BigDecimal[] arr = mutObj.getField("bdArr");
-        arr[0] = new BigDecimal(2000);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new BigDecimal[] {new BigDecimal(1000), new BigDecimal(1000), new BigDecimal(1000)},
-            res.bdArr);
-    }
-
-    /**
-     *
-     */
-    public void testBooleanArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.zArr = new boolean[] {false, false, false};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        boolean[] arr = mutObj.getField("zArr");
-        arr[0] = true;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        boolean[] expected = new boolean[] {true, false, false};
-
-        assertEquals(expected.length, res.zArr.length);
-
-        for (int i = 0; i < expected.length; i++)
-            assertEquals(expected[i], res.zArr[i]);
-    }
-
-    /**
-     *
-     */
-    public void testCharArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.cArr = new char[] {'a', 'a', 'a'};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        char[] arr = mutObj.getField("cArr");
-        arr[0] = 'b';
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new char[] {'b', 'a', 'a'}, res.cArr);
-    }
-
-    /**
-     *
-     */
-    public void testDoubleArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.dArr = new double[] {1.0, 1.0, 1.0};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        double[] arr = mutObj.getField("dArr");
-        arr[0] = 2.0;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new double[] {2.0, 1.0, 1.0}, res.dArr, 0);
-    }
-
-    /**
-     *
-     */
-    public void testFloatArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.fArr = new float[] {1.0f, 1.0f, 1.0f};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        float[] arr = mutObj.getField("fArr");
-        arr[0] = 2.0f;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new float[] {2.0f, 1.0f, 1.0f}, res.fArr, 0);
-    }
-
-    /**
-     *
-     */
-    public void testLongArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.lArr = new long[] {1, 1, 1};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        long[] arr = mutObj.getField("lArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new long[] {2, 1, 1}, res.lArr);
-    }
-
-    /**
-     *
-     */
-    public void testIntArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.iArr = new int[] {1, 1, 1};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        int[] arr = mutObj.getField("iArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new int[] {2, 1, 1}, res.iArr);
-    }
-
-    /**
-     *
-     */
-    public void testShortArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.sArr = new short[] {1, 1, 1};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        short[] arr = mutObj.getField("sArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new short[] {2, 1, 1}, res.sArr);
-    }
-
-    /**
-     *
-     */
-    public void testByteArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.bArr = new byte[] {1, 1, 1};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        byte[] arr = mutObj.getField("bArr");
-        arr[0] = 2;
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new byte[] {2, 1, 1}, res.bArr);
-    }
-
-    /**
-     *
-     */
-    public void testStringArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.strArr = new String[] {"a", "a", "a"};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        String[] arr = mutObj.getField("strArr");
-        arr[0] = "b";
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new String[] {"b", "a", "a"}, res.strArr);
-    }
-
-    /**
-     *
-     */
-    public void testModifyObjectArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {"a"};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = mutObj.getField("foo");
-
-        Assert.assertArrayEquals(new Object[] {"a"}, arr);
-
-        arr[0] = "b";
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new Object[] {"b"}, (Object[])res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testOverrideObjectArrayField() {
-        PortableBuilderImpl mutObj = wrap(new TestObjectContainer());
-
-        Object[] createdArr = {mutObj, "a", 1, new String[] {"s", "s"}, new byte[] {1, 2}, new UUID(3, 0)};
-
-        mutObj.setField("foo", createdArr.clone());
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        createdArr[0] = res;
-
-        assertTrue(Objects.deepEquals(createdArr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testDeepArray() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = new Object[] {new Object[] {"a", obj}};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Object[] arr = (Object[])mutObj.<Object[]>getField("foo")[0];
-
-        assertEquals("a", arr[0]);
-        assertSame(mutObj, arr[1]);
-
-        arr[0] = mutObj;
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        arr = (Object[])((Object[])res.foo)[0];
-
-        assertSame(arr[0], res);
-        assertSame(arr[0], arr[1]);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(obj, "a");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newArrayList(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testArrayListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        ArrayList<Object> list = Lists.newArrayList(mutObj, "a", Lists.newArrayList(1, 2));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof ArrayList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListClear() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.clear();
-
-        assertEquals(Collections.emptyList(), mutObj.build().<TestObjectContainer>deserialize().foo);
-    }
-
-    /**
-     *
-     */
-    public void testArrayListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        ArrayList<Object> src = Lists.newArrayList(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newLinkedList(Arrays.asList(obj, "a"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = mutObj.getField("foo");
-
-        assert list.equals(Lists.newLinkedList(Arrays.asList(mutObj, "a")));
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<Object> list = Lists.newLinkedList(Arrays.asList(mutObj, "a", Lists.newLinkedList(Arrays.asList(1, 2))));
-
-        mutObj.setField("foo", list);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        list.set(0, res);
-
-        assertNotSame(list, res.foo);
-        assertEquals(list, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newLinkedList(Arrays.asList("a", "b", "c"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        List<String> list = mutObj.getField("foo");
-
-        list.add("!"); // "a", "b", "c", "!"
-        list.add(0, "_"); // "_", "a", "b", "c", "!"
-
-        String s = list.remove(1); // "_", "b", "c", "!"
-        assertEquals("a", s);
-
-        assertEquals(Arrays.asList("c", "!"), list.subList(2, 4));
-        assertEquals(1, list.indexOf("b"));
-        assertEquals(1, list.lastIndexOf("b"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof LinkedList);
-        assertEquals(Arrays.asList("_", "b", "c", "!"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testLinkedListWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        LinkedList<Object> src = Lists.newLinkedList(Arrays.asList(obj, "a", "b", "c"));
-
-        obj.foo = src;
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        TestObjectContainer deserialized = mutObj.build().deserialize();
-
-        List<Object> res = (List<Object>)deserialized.foo;
-
-        src.set(0, deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet(obj, "a");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> set = mutObj.getField("foo");
-
-        assert set.equals(Sets.newHashSet(mutObj, "a"));
-    }
-
-    /**
-     *
-     */
-    public void testHashSetOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<Object> c = Sets.newHashSet(mutObj, "a", Sets.newHashSet(1, 2));
-
-        mutObj.setField("foo", c);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        c.remove(mutObj);
-        c.add(res);
-
-        assertNotSame(c, res.foo);
-        assertEquals(c, res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Sets.newHashSet("a", "b", "c");
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Set<String> set = mutObj.getField("foo");
-
-        set.remove("b");
-        set.add("!");
-
-        assertEquals(Sets.newHashSet("a", "!", "c"), set);
-        assertTrue(set.contains("a"));
-        assertTrue(set.contains("!"));
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertTrue(res.foo instanceof HashSet);
-        assertEquals(Sets.newHashSet("a", "!", "c"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testHashSetWriteUnmodifiable() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Set<Object> src = Sets.newHashSet(obj, "a", "b", "c");
-
-        obj.foo = src;
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        Set<Object> res = (Set<Object>)deserialized.foo;
-
-        src.remove(obj);
-        src.add(deserialized);
-
-        assertEquals(src, res);
-    }
-
-    /**
-     *
-     */
-    public void testMapRead() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(obj, "a", "b", obj));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        assert map.equals(ImmutableMap.of(mutObj, "a", "b", mutObj));
-    }
-
-    /**
-     *
-     */
-    public void testMapOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = Maps.newHashMap(ImmutableMap.of(mutObj, "a", "b", mutObj));
-
-        mutObj.setField("foo", map);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(res, "a", "b", res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b"));
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        Map<Object, Object> map = mutObj.getField("foo");
-
-        map.put(3, mutObj);
-        Object rmv = map.remove(1);
-
-        assertEquals("a", rmv);
-
-        TestObjectContainer res = mutObj.build().deserialize();
-
-        assertEquals(ImmutableMap.of(2, "b", 3, res), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testEnumArrayModification() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-
-        obj.enumArr = new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B};
-
-        PortableBuilderImpl mutObj = wrap(obj);
-
-        PortableBuilderEnum[] arr = mutObj.getField("enumArr");
-        arr[0] = new PortableBuilderEnum(mutObj.typeId(), TestObjectEnum.B);
-
-        TestObjectAllTypes res = mutObj.build().deserialize();
-
-        Assert.assertArrayEquals(new TestObjectEnum[] {TestObjectEnum.A, TestObjectEnum.B}, res.enumArr);
-    }
-
-    /**
-     *
-     */
-    public void testEditObjectWithRawData() {
-        GridPortableMarshalerAwareTestClass obj = new GridPortableMarshalerAwareTestClass();
-
-        obj.s = "a";
-        obj.sRaw = "aa";
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("s", "z");
-
-        GridPortableMarshalerAwareTestClass res = mutableObj.build().deserialize();
-        assertEquals("z", res.s);
-        assertEquals("aa", res.sRaw);
-    }
-
-    /**
-     *
-     */
-    public void testHashCode() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(obj.hashCode(), mutableObj.build().hashCode());
-
-        mutableObj.hashCode(25);
-
-        assertEquals(25, mutableObj.build().hashCode());
-    }
-
-    /**
-     *
-     */
-    public void testCollectionsInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = Lists.newArrayList(
-            Lists.newArrayList(1, 2),
-            Lists.newLinkedList(Arrays.asList(1, 2)),
-            Sets.newHashSet("a", "b"),
-            Sets.newLinkedHashSet(Arrays.asList("a", "b")),
-            Maps.newHashMap(ImmutableMap.of(1, "a", 2, "b")));
-
-        TestObjectContainer deserialized = wrap(obj).build().deserialize();
-
-        assertEquals(obj.foo, deserialized.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryModification() {
-        TestObjectContainer obj = new TestObjectContainer();
-        obj.foo = ImmutableMap.of(1, "a").entrySet().iterator().next();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        Map.Entry<Object, Object> entry = mutableObj.getField("foo");
-
-        assertEquals(1, entry.getKey());
-        assertEquals("a", entry.getValue());
-
-        entry.setValue("b");
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "b"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMapEntryOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        mutableObj.setField("foo", new GridMapEntry<>(1, "a"));
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(new GridMapEntry<>(1, "a"), res.foo);
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut() {
-        PortableBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChangingDoublePut2() {
-        PortableBuilderImpl mutableObj = wrap(new TestObjectContainer());
-
-        mutableObj.setField("xx567", "a");
-        mutableObj.setField("xx567", "b");
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(TestObjectContainer.class);
-
-        assertEquals("String", metadata.fieldTypeName("xx567"));
-    }
-
-    /**
-     *
-     */
-    public void testMetadataChanging() {
-        TestObjectContainer c = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(c);
-
-        mutableObj.setField("intField", 1);
-        mutableObj.setField("intArrField", new int[] {1});
-        mutableObj.setField("arrField", new String[] {"1"});
-        mutableObj.setField("strField", "1");
-        mutableObj.setField("colField", Lists.newArrayList("1"));
-        mutableObj.setField("mapField", Maps.newHashMap(ImmutableMap.of(1, "1")));
-        mutableObj.setField("enumField", TestObjectEnum.A);
-        mutableObj.setField("enumArrField", new Enum[] {TestObjectEnum.A});
-
-        mutableObj.build();
-
-        PortableMetadata metadata = portables().metadata(c.getClass());
-
-        assertTrue(metadata.fields().containsAll(Arrays.asList("intField", "intArrField", "arrField", "strField",
-            "colField", "mapField", "enumField", "enumArrField")));
-
-        assertEquals("int", metadata.fieldTypeName("intField"));
-        assertEquals("int[]", metadata.fieldTypeName("intArrField"));
-        assertEquals("String[]", metadata.fieldTypeName("arrField"));
-        assertEquals("String", metadata.fieldTypeName("strField"));
-        assertEquals("Collection", metadata.fieldTypeName("colField"));
-        assertEquals("Map", metadata.fieldTypeName("mapField"));
-        assertEquals("Enum", metadata.fieldTypeName("enumField"));
-        assertEquals("Enum[]", metadata.fieldTypeName("enumArrField"));
-    }
-
-    /**
-     *
-     */
-    public void testDateInObjectField() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = new Date();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Date.class, mutableObj.getField("foo").getClass());
-    }
-
-    /**
-     *
-     */
-    public void testTimestampInObjectField() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = new Timestamp(100020003);
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, mutableObj.getField("foo").getClass());
-    }
-
-    /**
-     *
-     */
-    public void testDateInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newArrayList(new Date());
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Date.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
-    }
-
-    /**
-     *
-     */
-    public void testTimestampInCollection() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        obj.foo = Lists.newArrayList(new Timestamp(100020003));
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        assertEquals(Timestamp.class, ((List<?>)mutableObj.getField("foo")).get(0).getClass());
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
-    public void testDateArrayOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        Date[] arr = { new Date() };
-
-        mutableObj.setField("foo", arr);
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(Date[].class, res.foo.getClass());
-        assertTrue(Objects.deepEquals(arr, res.foo));
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("AssertEqualsBetweenInconvertibleTypes")
-    public void testTimestampArrayOverride() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        PortableBuilderImpl mutableObj = wrap(obj);
-
-        Timestamp[] arr = { new Timestamp(100020003) };
-
-        mutableObj.setField("foo", arr);
-
-        TestObjectContainer res = mutableObj.build().deserialize();
-
-        assertEquals(Timestamp[].class, res.foo.getClass());
-        assertTrue(Objects.deepEquals(arr, res.foo));
-    }
-
-    /**
-     *
-     */
-    public void testChangeMap() {
-        AddressBook addrBook = new AddressBook();
-
-        addrBook.addCompany(new Company(1, "Google inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 53), "occupation"));
-        addrBook.addCompany(new Company(2, "Apple inc", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 54), "occupation"));
-        addrBook.addCompany(new Company(3, "Microsoft", 100, new Address("Saint-Petersburg", "Torzhkovskya", 1, 55), "occupation"));
-        addrBook.addCompany(new Company(4, "Oracle", 100, new Address("Saint-Petersburg", "Nevskiy", 1, 1), "occupation"));
-
-        PortableBuilderImpl mutableObj = wrap(addrBook);
-
-        Map<String, List<PortableBuilderImpl>> map = mutableObj.getField("companyByStreet");
-
-        List<PortableBuilderImpl> list = map.get("Torzhkovskya");
-
-        PortableBuilderImpl company = list.get(0);
-
-        assert "Google inc".equals(company.<String>getField("name"));
-
-        list.remove(0);
-
-        AddressBook res = mutableObj.build().deserialize();
-
-        assertEquals(Arrays.asList("Nevskiy", "Torzhkovskya"), new ArrayList<>(res.getCompanyByStreet().keySet()));
-
-        List<Company> torzhkovskyaCompanies = res.getCompanyByStreet().get("Torzhkovskya");
-
-        assertEquals(2, torzhkovskyaCompanies.size());
-        assertEquals("Apple inc", torzhkovskyaCompanies.get(0).name);
-    }
-
-    /**
-     *
-     */
-    public void testSavingObjectWithNotZeroStart() {
-        TestObjectOuter out = new TestObjectOuter();
-        TestObjectInner inner = new TestObjectInner();
-
-        out.inner = inner;
-        inner.outer = out;
-
-        PortableBuilderImpl builder = wrap(out);
-
-        PortableBuilderImpl innerBuilder = builder.getField("inner");
-
-        TestObjectInner res = innerBuilder.build().deserialize();
-
-        assertSame(res, res.outer.inner);
-    }
-
-    /**
-     *
-     */
-    public void testPortableObjectField() {
-        TestObjectContainer container = new TestObjectContainer(toPortable(new TestObjectArrayList()));
-
-        PortableBuilderImpl wrapper = wrap(container);
-
-        assertTrue(wrapper.getField("foo") instanceof PortableObject);
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof PortableObject);
-    }
-
-    /**
-     *
-     */
-    public void testAssignPortableObject() {
-        TestObjectContainer container = new TestObjectContainer();
-
-        PortableBuilderImpl wrapper = wrap(container);
-
-        wrapper.setField("foo", toPortable(new TestObjectArrayList()));
-
-        TestObjectContainer deserialized = wrapper.build().deserialize();
-        assertTrue(deserialized.foo instanceof TestObjectArrayList);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromNewObject() {
-        PortableBuilderImpl wrapper = newWrapper(TestObjectAllTypes.class);
-
-        wrapper.setField("str", "a");
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testRemoveFromExistingObject() {
-        TestObjectAllTypes obj = new TestObjectAllTypes();
-        obj.setDefaultData();
-
-        PortableBuilderImpl wrapper = wrap(toPortable(obj));
-
-        wrapper.removeField("str");
-
-        assertNull(wrapper.build().<TestObjectAllTypes>deserialize().str);
-    }
-
-    /**
-     *
-     */
-    public void testCyclicArrays() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        Object[] arr1 = new Object[1];
-        Object[] arr2 = new Object[] {arr1};
-
-        arr1[0] = arr2;
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        Object[] resArr = (Object[])res.foo;
-
-        assertSame(((Object[])resArr[0])[0], resArr);
-    }
-
-    /**
-     *
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    public void testCyclicArrayList() {
-        TestObjectContainer obj = new TestObjectContainer();
-
-        List<Object> arr1 = new ArrayList<>();
-        List<Object> arr2 = new ArrayList<>();
-
-        arr1.add(arr2);
-        arr2.add(arr1);
-
-        obj.foo = arr1;
-
-        TestObjectContainer res = toPortable(obj).deserialize();
-
-        List<?> resArr = (List<?>)res.foo;
-
-        assertSame(((List<Object>)resArr.get(0)).get(0), resArr);
-    }
-
-    /**
-     * @param obj Object.
-     * @return Object in portable format.
-     */
-    private PortableObject toPortable(Object obj) {
-        return portables().toPortable(obj);
-    }
-
-    /**
-     * @param obj Object.
-     * @return GridMutablePortableObject.
-     */
-    private PortableBuilderImpl wrap(Object obj) {
-        return PortableBuilderImpl.wrap(toPortable(obj));
-    }
-
-    /**
-     * @param aCls Class.
-     * @return Wrapper.
-     */
-    private PortableBuilderImpl newWrapper(Class<?> aCls) {
-        CacheObjectPortableProcessorImpl processor = (CacheObjectPortableProcessorImpl)(
-            (IgnitePortablesImpl)portables()).processor();
-
-        return new PortableBuilderImpl(processor.portableContext(), processor.typeId(aCls.getName()),
-            aCls.getSimpleName());
-    }
-}
\ No newline at end of file


[09/19] ignite git commit: ignite-950-new WIP

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
index c86de5d..f8ab6f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
@@ -21,8 +21,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.store.CacheStore;
 import org.apache.ignite.cache.store.CacheStoreSession;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStore;
 import org.apache.ignite.internal.processors.platform.cache.store.PlatformCacheStoreCallback;
@@ -170,8 +170,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
         try {
             final GridTuple<V> val = new GridTuple<>();
 
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_LOAD);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -191,8 +191,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
         try {
             final Map<K, V> loaded = new HashMap<>();
 
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_LOAD_ALL);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -210,8 +210,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** {@inheritDoc} */
     @Override public void loadCache(final IgniteBiInClosure<K, V> clo, final @Nullable Object... args) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_LOAD_CACHE);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -227,8 +227,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** {@inheritDoc} */
     @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_PUT);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -246,8 +246,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     @SuppressWarnings({"NullableProblems", "unchecked"})
     @Override public void writeAll(final Collection<Cache.Entry<? extends K, ? extends V>> entries) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     Map<K, V> map = new AbstractMap<K, V>() {
                         @Override public int size() {
                             return entries.size();
@@ -287,8 +287,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** {@inheritDoc} */
     @Override public void delete(final Object key) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_RMV);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -304,8 +304,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** {@inheritDoc} */
     @Override public void deleteAll(final Collection<?> keys) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_RMV_ALL);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -321,8 +321,8 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
     /** {@inheritDoc} */
     @Override public void sessionEnd(final boolean commit) {
         try {
-            doInvoke(new IgniteInClosureX<PortableRawWriterEx>() {
-                @Override public void applyx(PortableRawWriterEx writer) throws IgniteCheckedException {
+            doInvoke(new IgniteInClosureX<IgniteObjectRawWriterEx>() {
+                @Override public void applyx(IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
                     writer.writeByte(OP_SES_END);
                     writer.writeLong(session());
                     writer.writeString(ses.cacheName());
@@ -351,7 +351,7 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeString(assemblyName);
             writer.writeString(clsName);
@@ -391,12 +391,12 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
      * @return Result.
      * @throws org.apache.ignite.IgniteCheckedException If failed.
      */
-    protected int doInvoke(IgniteInClosureX<PortableRawWriterEx> task, @Nullable PlatformCacheStoreCallback cb)
+    protected int doInvoke(IgniteInClosureX<IgniteObjectRawWriterEx> task, @Nullable PlatformCacheStoreCallback cb)
         throws IgniteCheckedException{
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             task.apply(writer);
 
@@ -438,7 +438,7 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
 
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override protected void invoke0(PortableRawReaderEx reader) {
+        @Override protected void invoke0(IgniteObjectRawReaderEx reader) {
             val.set((V)reader.readObjectDetached());
         }
     }
@@ -464,7 +464,7 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
 
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override protected void invoke0(PortableRawReaderEx reader) {
+        @Override protected void invoke0(IgniteObjectRawReaderEx reader) {
             loaded.put((K) reader.readObjectDetached(), (V) reader.readObjectDetached());
         }
     }
@@ -490,7 +490,7 @@ public class PlatformDotNetCacheStore<K, V> implements CacheStore<K, V>, Platfor
 
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override protected void invoke0(PortableRawReaderEx reader) {
+        @Override protected void invoke0(IgniteObjectRawReaderEx reader) {
             clo.apply((K) reader.readObjectDetached(), (V) reader.readObjectDetached());
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
index 35d2df4..8513650 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetConfigurationClosure.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.MarshallerContextImpl;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.internal.portable.PortableMetaDataHandler;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure;
 import org.apache.ignite.internal.processors.platform.lifecycle.PlatformLifecycleBean;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
@@ -39,8 +39,8 @@ import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.marshaller.portable.PortableMarshaller;
 import org.apache.ignite.platform.dotnet.PlatformDotNetLifecycleBean;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectMetadata;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -137,7 +137,7 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
             try (PlatformMemory inMem = memMgr.allocate()) {
                 PlatformOutputStream out = outMem.output();
 
-                PortableRawWriterEx writer = marshaller().writer(out);
+                IgniteObjectRawWriterEx writer = marshaller().writer(out);
 
                 PlatformUtils.writeDotNetConfiguration(writer, interopCfg.unwrap());
 
@@ -230,12 +230,12 @@ public class PlatformDotNetConfigurationClosure extends PlatformAbstractConfigur
     private static GridPortableMarshaller marshaller() {
         try {
             PortableContext ctx = new PortableContext(new PortableMetaDataHandler() {
-                @Override public void addMeta(int typeId, PortableMetadata meta)
-                    throws PortableException {
+                @Override public void addMeta(int typeId, IgniteObjectMetadata meta)
+                    throws IgniteObjectException {
                     // No-op.
                 }
 
-                @Override public PortableMetadata metadata(int typeId) throws PortableException {
+                @Override public IgniteObjectMetadata metadata(int typeId) throws IgniteObjectException {
                     return null;
                 }
             }, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
index b2dfd1c..17064d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEventFilterListenerImpl.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.events;
 
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -114,7 +114,7 @@ public class PlatformEventFilterListenerImpl implements PlatformEventFilterListe
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             ctx.writeEvent(writer, evt);
 
@@ -140,7 +140,7 @@ public class PlatformEventFilterListenerImpl implements PlatformEventFilterListe
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObjectDetached(pred);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
index 8585526..37bfcae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -24,8 +24,8 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventAdapter;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
@@ -137,7 +137,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_RECORD_LOCAL:
@@ -168,7 +168,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_LOCAL_QUERY: {
@@ -256,7 +256,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_ENABLED_EVENTS:
                 writeEventTypes(events.enabledEvents(), writer);
@@ -292,7 +292,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
      * @param reader Reader
      * @return Event types, or null.
      */
-    private int[] readEventTypes(PortableRawReaderEx reader) {
+    private int[] readEventTypes(IgniteObjectRawReaderEx reader) {
         return reader.readIntArray();
     }
 
@@ -302,7 +302,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
      * @param writer Writer
      * @param types Types.
      */
-    private void writeEventTypes(int[] types, PortableRawWriterEx writer) {
+    private void writeEventTypes(int[] types, IgniteObjectRawWriterEx writer) {
         if (types == null) {
             writer.writeIntArray(null);
 
@@ -349,7 +349,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
         }
 
         /** <inheritDoc /> */
-        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+        @Override public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err) {
             platformCtx.writeEvent(writer, (EventAdapter)obj);
         }
 
@@ -379,7 +379,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
 
         /** <inheritDoc /> */
         @SuppressWarnings("unchecked")
-        @Override public void write(PortableRawWriterEx writer, Object obj, Throwable err) {
+        @Override public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err) {
             Collection<EventAdapter> events = (Collection<EventAdapter>)obj;
 
             writer.writeInt(events.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
index 67d5bbb..fdc3460 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageFilterImpl.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.messaging;
 
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -62,7 +62,7 @@ public class PlatformMessageFilterImpl extends PlatformAbstractPredicate impleme
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(uuid);
             writer.writeObject(m);
@@ -83,7 +83,7 @@ public class PlatformMessageFilterImpl extends PlatformAbstractPredicate impleme
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             writer.writeObject(pred);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
index 50643e1..4ee7969 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessageLocalFilter.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.messaging;
 
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -58,7 +58,7 @@ public class PlatformMessageLocalFilter implements PlatformMessageFilter {
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeObject(uuid);
             writer.writeObject(m);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
index 6dfd570..dbbf5da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.processors.platform.messaging;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
@@ -84,7 +84,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_SEND:
@@ -135,7 +135,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_REMOTE_LISTEN:{

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
index 0b9ee53..6d6cf06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformAbstractService.java
@@ -23,8 +23,8 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
@@ -84,7 +84,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeBoolean(srvKeepPortable);
             writer.writeObject(svc);
@@ -108,7 +108,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeBoolean(srvKeepPortable);
 
@@ -131,7 +131,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
         try (PlatformMemory mem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeBoolean(srvKeepPortable);
 
@@ -152,7 +152,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
      * @param ctx Context.
      * @param writer Writer.
      */
-    private void writeServiceContext(ServiceContext ctx, PortableRawWriterEx writer) {
+    private void writeServiceContext(ServiceContext ctx, IgniteObjectRawWriterEx writer) {
         writer.writeString(ctx.name());
         writer.writeUuid(ctx.executionId());
         writer.writeBoolean(ctx.isCancelled());
@@ -175,7 +175,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
 
         try (PlatformMemory outMem = platformCtx.memory().allocate()) {
             PlatformOutputStream out = outMem.output();
-            PortableRawWriterEx writer = platformCtx.writer(out);
+            IgniteObjectRawWriterEx writer = platformCtx.writer(out);
 
             writer.writeBoolean(srvKeepPortable);
             writer.writeString(mthdName);
@@ -195,7 +195,7 @@ public abstract class PlatformAbstractService implements PlatformService, Extern
             try (PlatformMemory inMem = platformCtx.memory().allocate()) {
                 PlatformInputStream in = inMem.input();
 
-                PortableRawReaderEx reader = platformCtx.reader(in);
+                IgniteObjectRawReaderEx reader = platformCtx.reader(in);
 
                 platformCtx.gateway().serviceInvokeMethod(ptr, outMem.pointer(), inMem.pointer());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index bfa4570..b6efb0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -22,8 +22,8 @@ import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteServices;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService;
@@ -128,7 +128,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, PortableRawReaderEx reader)
+    @Override protected long processInStreamOutLong(int type, IgniteObjectRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_DEPLOY: {
@@ -169,7 +169,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, PortableRawReaderEx reader, PortableRawWriterEx writer)
+    @Override protected void processInStreamOutStream(int type, IgniteObjectRawReaderEx reader, IgniteObjectRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_SERVICES: {
@@ -177,7 +177,7 @@ public class PlatformServices extends PlatformAbstractTarget {
 
                 PlatformUtils.writeNullableCollection(writer, svcs,
                     new PlatformWriterClosure<Service>() {
-                        @Override public void write(PortableRawWriterEx writer, Service svc) {
+                        @Override public void write(IgniteObjectRawWriterEx writer, Service svc) {
                             writer.writeLong(((PlatformService) svc).pointer());
                         }
                     },
@@ -197,8 +197,8 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInObjectStreamOutStream(int type, Object arg, PortableRawReaderEx reader,
-        PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processInObjectStreamOutStream(int type, Object arg, IgniteObjectRawReaderEx reader,
+        IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_INVOKE: {
                 assert arg != null;
@@ -235,13 +235,13 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_DESCRIPTORS: {
                 Collection<ServiceDescriptor> descs = services.serviceDescriptors();
 
                 PlatformUtils.writeCollection(writer, descs, new PlatformWriterClosure<ServiceDescriptor>() {
-                    @Override public void write(PortableRawWriterEx writer, ServiceDescriptor d) {
+                    @Override public void write(IgniteObjectRawWriterEx writer, ServiceDescriptor d) {
                         writer.writeString(d.name());
                         writer.writeString(d.cacheName());
                         writer.writeInt(d.maxPerNodeCount());
@@ -252,7 +252,7 @@ public class PlatformServices extends PlatformAbstractTarget {
                         Map<UUID, Integer> top = d.topologySnapshot();
 
                         PlatformUtils.writeMap(writer, top, new PlatformWriterBiClosure<UUID, Integer>() {
-                            @Override public void write(PortableRawWriterEx writer, UUID key, Integer val) {
+                            @Override public void write(IgniteObjectRawWriterEx writer, UUID key, Integer val) {
                                 writer.writeUuid(key);
                                 writer.writeInt(val);
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index c143212..3fdfddf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -18,13 +18,12 @@
 package org.apache.ignite.internal.processors.platform.transactions;
 
 import java.sql.Timestamp;
-import java.util.Date;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteTransactions;
 import org.apache.ignite.configuration.TransactionConfiguration;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
@@ -232,7 +231,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, PortableRawWriterEx writer) throws IgniteCheckedException {
+    @Override protected void processOutStream(int type, IgniteObjectRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_CACHE_CONFIG_PARAMETERS:
                 TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
index 0019986..7cf8adb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.platform.utils;
 
 import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
@@ -164,7 +164,7 @@ public class PlatformFutureUtils {
                             try (PlatformMemory mem = ctx.memory().allocate()) {
                                 PlatformOutputStream out = mem.output();
 
-                                PortableRawWriterEx outWriter = ctx.writer(out);
+                                IgniteObjectRawWriterEx outWriter = ctx.writer(out);
 
                                 outWriter.writeObjectDetached(res);
 
@@ -244,7 +244,7 @@ public class PlatformFutureUtils {
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx outWriter = ctx.writer(out);
+            IgniteObjectRawWriterEx outWriter = ctx.writer(out);
 
             outWriter.writeString(err.getClass().getName());
             outWriter.writeString(err.getMessage());
@@ -276,7 +276,7 @@ public class PlatformFutureUtils {
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx outWriter = ctx.writer(out);
+            IgniteObjectRawWriterEx outWriter = ctx.writer(out);
 
             writer.write(outWriter, obj, err);
 
@@ -299,7 +299,7 @@ public class PlatformFutureUtils {
          * @param obj Object.
          * @param err Error.
          */
-        public void write(PortableRawWriterEx writer, Object obj, Throwable err);
+        public void write(IgniteObjectRawWriterEx writer, Object obj, Throwable err);
 
         /**
          * Determines whether this writer can write given data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
index 79759e9..4bda62a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderBiClosure.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 import org.apache.ignite.lang.IgniteBiTuple;
 
 /**
@@ -30,5 +30,5 @@ public interface PlatformReaderBiClosure<T1, T2> {
      * @param reader Reader.
      * @return Object.
      */
-    IgniteBiTuple<T1, T2> read(PortableRawReaderEx reader);
+    IgniteBiTuple<T1, T2> read(IgniteObjectRawReaderEx reader);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
index 2d9b44a..3bec262 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformReaderClosure.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
 
 /**
  * Reader closure.
@@ -30,5 +30,5 @@ public interface PlatformReaderClosure<T> {
      * @param reader Reader.
      * @return Object.
      */
-    T read(PortableRawReaderEx reader);
+    T read(IgniteObjectRawReaderEx reader);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
index 11d8371..a1d1cae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -24,8 +24,8 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.portable.PortableRawReaderEx;
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawReaderEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformExtendedException;
 import org.apache.ignite.internal.processors.platform.PlatformNativeException;
@@ -96,7 +96,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param col Collection to write.
      */
-    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col) {
+    public static <T> void writeNullableCollection(IgniteObjectRawWriterEx writer, @Nullable Collection<T> col) {
         writeNullableCollection(writer, col, null, null);
     }
 
@@ -107,7 +107,7 @@ public class PlatformUtils {
      * @param col Collection to write.
      * @param writeClo Writer closure.
      */
-    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col,
+    public static <T> void writeNullableCollection(IgniteObjectRawWriterEx writer, @Nullable Collection<T> col,
         @Nullable PlatformWriterClosure<T> writeClo) {
         writeNullableCollection(writer, col, writeClo, null);
     }
@@ -120,7 +120,7 @@ public class PlatformUtils {
      * @param writeClo Optional writer closure.
      * @param filter Optional filter.
      */
-    public static <T> void writeNullableCollection(PortableRawWriterEx writer, @Nullable Collection<T> col,
+    public static <T> void writeNullableCollection(IgniteObjectRawWriterEx writer, @Nullable Collection<T> col,
         @Nullable PlatformWriterClosure<T> writeClo, @Nullable IgnitePredicate<T> filter) {
         if (col != null) {
             writer.writeBoolean(true);
@@ -137,7 +137,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param col Collection to write.
      */
-    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col) {
+    public static <T> void writeCollection(IgniteObjectRawWriterEx writer, Collection<T> col) {
         writeCollection(writer, col, null, null);
     }
 
@@ -148,7 +148,7 @@ public class PlatformUtils {
      * @param col Collection to write.
      * @param writeClo Writer closure.
      */
-    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col,
+    public static <T> void writeCollection(IgniteObjectRawWriterEx writer, Collection<T> col,
         @Nullable PlatformWriterClosure<T> writeClo) {
         writeCollection(writer, col, writeClo, null);
     }
@@ -161,7 +161,7 @@ public class PlatformUtils {
      * @param writeClo Optional writer closure.
      * @param filter Optional filter.
      */
-    public static <T> void writeCollection(PortableRawWriterEx writer, Collection<T> col,
+    public static <T> void writeCollection(IgniteObjectRawWriterEx writer, Collection<T> col,
         @Nullable PlatformWriterClosure<T> writeClo, @Nullable IgnitePredicate<T> filter) {
         assert col != null;
 
@@ -202,7 +202,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param map Map to write.
      */
-    public static <K, V> void writeNullableMap(PortableRawWriterEx writer, @Nullable Map<K, V> map) {
+    public static <K, V> void writeNullableMap(IgniteObjectRawWriterEx writer, @Nullable Map<K, V> map) {
         if (map != null) {
             writer.writeBoolean(true);
 
@@ -218,7 +218,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param map Map to write.
      */
-    public static <K, V> void writeMap(PortableRawWriterEx writer, Map<K, V> map) {
+    public static <K, V> void writeMap(IgniteObjectRawWriterEx writer, Map<K, V> map) {
         assert map != null;
 
         writeMap(writer, map, null);
@@ -231,7 +231,7 @@ public class PlatformUtils {
      * @param map Map to write.
      * @param writeClo Writer closure.
      */
-    public static <K, V> void writeMap(PortableRawWriterEx writer, Map<K, V> map,
+    public static <K, V> void writeMap(IgniteObjectRawWriterEx writer, Map<K, V> map,
         @Nullable PlatformWriterBiClosure<K, V> writeClo) {
         assert map != null;
 
@@ -255,7 +255,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return List.
      */
-    public static <T> List<T> readCollection(PortableRawReaderEx reader) {
+    public static <T> List<T> readCollection(IgniteObjectRawReaderEx reader) {
         return readCollection(reader, null);
     }
 
@@ -266,7 +266,7 @@ public class PlatformUtils {
      * @param readClo Optional reader closure.
      * @return List.
      */
-    public static <T> List<T> readCollection(PortableRawReaderEx reader, @Nullable PlatformReaderClosure<T> readClo) {
+    public static <T> List<T> readCollection(IgniteObjectRawReaderEx reader, @Nullable PlatformReaderClosure<T> readClo) {
         int cnt = reader.readInt();
 
         List<T> res = new ArrayList<>(cnt);
@@ -289,7 +289,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return List.
      */
-    public static <T> List<T> readNullableCollection(PortableRawReaderEx reader) {
+    public static <T> List<T> readNullableCollection(IgniteObjectRawReaderEx reader) {
         return readNullableCollection(reader, null);
     }
 
@@ -299,7 +299,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return List.
      */
-    public static <T> List<T> readNullableCollection(PortableRawReaderEx reader,
+    public static <T> List<T> readNullableCollection(IgniteObjectRawReaderEx reader,
         @Nullable PlatformReaderClosure<T> readClo) {
         if (!reader.readBoolean())
             return null;
@@ -311,7 +311,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return Set.
      */
-    public static <T> Set<T> readSet(PortableRawReaderEx reader) {
+    public static <T> Set<T> readSet(IgniteObjectRawReaderEx reader) {
         int cnt = reader.readInt();
 
         Set<T> res = U.newHashSet(cnt);
@@ -326,7 +326,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return Set.
      */
-    public static <T> Set<T> readNullableSet(PortableRawReaderEx reader) {
+    public static <T> Set<T> readNullableSet(IgniteObjectRawReaderEx reader) {
         if (!reader.readBoolean())
             return null;
 
@@ -339,7 +339,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return Map.
      */
-    public static <K, V> Map<K, V> readMap(PortableRawReaderEx reader) {
+    public static <K, V> Map<K, V> readMap(IgniteObjectRawReaderEx reader) {
         return readMap(reader, null);
     }
 
@@ -350,7 +350,7 @@ public class PlatformUtils {
      * @param readClo Reader closure.
      * @return Map.
      */
-    public static <K, V> Map<K, V> readMap(PortableRawReaderEx reader,
+    public static <K, V> Map<K, V> readMap(IgniteObjectRawReaderEx reader,
         @Nullable PlatformReaderBiClosure<K, V> readClo) {
         int cnt = reader.readInt();
 
@@ -377,7 +377,7 @@ public class PlatformUtils {
      * @param reader Reader.
      * @return Map.
      */
-    public static <K, V> Map<K, V> readNullableMap(PortableRawReaderEx reader) {
+    public static <K, V> Map<K, V> readNullableMap(IgniteObjectRawReaderEx reader) {
         if (!reader.readBoolean())
             return null;
 
@@ -390,7 +390,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param val Values.
      */
-    public static void writeIgniteUuid(PortableRawWriterEx writer, IgniteUuid val) {
+    public static void writeIgniteUuid(IgniteObjectRawWriterEx writer, IgniteUuid val) {
         if (val == null)
             writer.writeUuid(null);
         else {
@@ -483,7 +483,7 @@ public class PlatformUtils {
         try (PlatformMemory mem = ctx.memory().allocate()) {
             PlatformOutputStream out = mem.output();
 
-            PortableRawWriterEx writer = ctx.writer(out);
+            IgniteObjectRawWriterEx writer = ctx.writer(out);
 
             int cntPos = writer.reserveInt();
 
@@ -552,7 +552,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param evt Event.
      */
-    private static void writeCacheEntryEvent(PortableRawWriterEx writer, CacheEntryEvent evt) {
+    private static void writeCacheEntryEvent(IgniteObjectRawWriterEx writer, CacheEntryEvent evt) {
         writer.writeObjectDetached(evt.getKey());
         writer.writeObjectDetached(evt.getOldValue());
         writer.writeObjectDetached(evt.getValue());
@@ -564,7 +564,7 @@ public class PlatformUtils {
      * @param err Error.
      * @param writer Writer.
      */
-    public static void writeErrorData(Throwable err, PortableRawWriterEx writer) {
+    public static void writeErrorData(Throwable err, IgniteObjectRawWriterEx writer) {
         writeErrorData(err, writer, null);
     }
 
@@ -574,7 +574,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param log Optional logger.
      */
-    public static void writeErrorData(Throwable err, PortableRawWriterEx writer, @Nullable IgniteLogger log) {
+    public static void writeErrorData(Throwable err, IgniteObjectRawWriterEx writer, @Nullable IgniteLogger log) {
         // Write additional data if needed.
         if (err instanceof PlatformExtendedException) {
             PlatformExtendedException err0 = (PlatformExtendedException)err;
@@ -659,7 +659,7 @@ public class PlatformUtils {
                 // Write error data.
                 PlatformOutputStream out = mem.output();
 
-                PortableRawWriterEx writer = ctx.writer(out);
+                IgniteObjectRawWriterEx writer = ctx.writer(out);
 
                 try {
                     PlatformUtils.writeErrorData(err, writer, ctx.kernalContext().log(PlatformContext.class));
@@ -696,7 +696,7 @@ public class PlatformUtils {
      * @param resObj Result.
      * @param err Error.
      */
-    public static void writeInvocationResult(PortableRawWriterEx writer, Object resObj, Exception err)
+    public static void writeInvocationResult(IgniteObjectRawWriterEx writer, Object resObj, Exception err)
     {
         if (err == null) {
             writer.writeBoolean(true);
@@ -732,7 +732,7 @@ public class PlatformUtils {
      * @return Result.
      * @throws IgniteCheckedException When invocation result is an error.
      */
-    public static Object readInvocationResult(PlatformContext ctx, PortableRawReaderEx reader)
+    public static Object readInvocationResult(PlatformContext ctx, IgniteObjectRawReaderEx reader)
         throws IgniteCheckedException {
         // 1. Read success flag.
         boolean success = reader.readBoolean();
@@ -769,7 +769,7 @@ public class PlatformUtils {
      * @param writer Writer.
      * @param cfg Configuration.
      */
-    public static void writeDotNetConfiguration(PortableRawWriterEx writer, PlatformDotNetConfiguration cfg) {
+    public static void writeDotNetConfiguration(IgniteObjectRawWriterEx writer, PlatformDotNetConfiguration cfg) {
         // 1. Write assemblies.
         writeNullableCollection(writer, cfg.getAssemblies());
 
@@ -780,7 +780,7 @@ public class PlatformUtils {
 
             writeNullableCollection(writer, portableCfg.getTypesConfiguration(),
                 new PlatformWriterClosure<PlatformDotNetPortableTypeConfiguration>() {
-                @Override public void write(PortableRawWriterEx writer, PlatformDotNetPortableTypeConfiguration typ) {
+                @Override public void write(IgniteObjectRawWriterEx writer, PlatformDotNetPortableTypeConfiguration typ) {
                     writer.writeString(typ.getAssemblyName());
                     writer.writeString(typ.getTypeName());
                     writer.writeString(typ.getNameMapper());

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
index 6b04ad3..c944345 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterBiClosure.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 
 /**
  * Interop writer bi-closure.
@@ -30,5 +30,5 @@ public interface PlatformWriterBiClosure<T1, T2> {
      * @param val1 Value 1.
      * @param val2 Value 2.
      */
-    public void write(PortableRawWriterEx writer, T1 val1, T2 val2);
+    public void write(IgniteObjectRawWriterEx writer, T1 val1, T2 val2);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
index a67d70a..2c9ab07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformWriterClosure.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
-import org.apache.ignite.internal.portable.PortableRawWriterEx;
+import org.apache.ignite.internal.portable.IgniteObjectRawWriterEx;
 
 /**
  * Interop writer closure.
@@ -29,5 +29,5 @@ public interface PlatformWriterClosure<T> {
      * @param writer Writer.
      * @param val Value.
      */
-    public void write(PortableRawWriterEx writer, T val);
+    public void write(IgniteObjectRawWriterEx writer, T val);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e26c140..2b54e2b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -46,7 +46,8 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.CacheTypeMetadata;
 import org.apache.ignite.cache.QueryEntity;
-import org.apache.ignite.cache.QueryEntityIndex;
+import org.apache.ignite.cache.QueryIndex;
+import org.apache.ignite.cache.QueryIndexType;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
@@ -1410,40 +1411,33 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (aliases == null)
                 aliases = Collections.emptyMap();
 
-            for (QueryEntityIndex idx : qryEntity.getIndexes()) {
+            for (QueryIndex idx : qryEntity.getIndexes()) {
                 String idxName = idx.getName();
 
                 if (idxName == null)
                     idxName = QueryEntity.defaultIndexName(idx);
 
-                if (idx.getType() == QueryEntityIndex.Type.SORTED || idx.getType() == QueryEntityIndex.Type.GEOSPATIAL) {
-                    d.addIndex(idxName, idx.getType() == QueryEntityIndex.Type.SORTED ? SORTED : GEO_SPATIAL);
+                if (idx.getIndexType() == QueryIndexType.SORTED || idx.getIndexType() == QueryIndexType.GEOSPATIAL) {
+                    d.addIndex(idxName, idx.getIndexType() == QueryIndexType.SORTED ? SORTED : GEO_SPATIAL);
 
                     int i = 0;
 
-                    for (String field : idx.getFields()) {
-                        boolean desc = false;
-
-                        int space = field.indexOf(' ');
-
-                        if (space != -1) {
-                            desc = field.toLowerCase().startsWith("desc", space + 1);
-
-                            field = field.substring(0, space);
-                        }
+                    for (Map.Entry<String, Boolean> entry : idx.getFields().entrySet()) {
+                        String field = entry.getKey();
+                        boolean asc = entry.getValue();
 
                         String alias = aliases.get(field);
 
                         if (alias != null)
                             field = alias;
 
-                        d.addFieldToIndex(idxName, field, i++, desc);
+                        d.addFieldToIndex(idxName, field, i++, !asc);
                     }
                 }
                 else {
-                    assert idx.getType() == QueryEntityIndex.Type.FULLTEXT;
+                    assert idx.getIndexType() == QueryIndexType.FULLTEXT;
 
-                    for (String field : idx.getFields()) {
+                    for (String field : idx.getFields().keySet()) {
                         String alias = aliases.get(field);
 
                         if (alias != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
index ea2e1cc..e44acfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/portable/PortableMarshaller.java
@@ -28,11 +28,11 @@ import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableContext;
 import org.apache.ignite.marshaller.AbstractMarshaller;
 import org.apache.ignite.marshaller.MarshallerContext;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.igniteobject.IgniteObject;
+import org.apache.ignite.igniteobject.IgniteObjectConfiguration;
+import org.apache.ignite.igniteobject.IgniteObjectException;
+import org.apache.ignite.igniteobject.IgniteObjectIdMapper;
+import org.apache.ignite.igniteobject.IgniteObjectSerializer;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -79,13 +79,13 @@ public class PortableMarshaller extends AbstractMarshaller {
     private Collection<String> clsNames;
 
     /** ID mapper. */
-    private PortableIdMapper idMapper;
+    private IgniteObjectIdMapper idMapper;
 
     /** Serializer. */
-    private PortableSerializer serializer;
+    private IgniteObjectSerializer serializer;
 
     /** Types. */
-    private Collection<PortableTypeConfiguration> typeCfgs;
+    private Collection<IgniteObjectConfiguration> typeCfgs;
 
     /** Whether to convert string to bytes using UTF-8 encoding. */
     private boolean convertString = true;
@@ -125,7 +125,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @return ID mapper.
      */
-    public PortableIdMapper getIdMapper() {
+    public IgniteObjectIdMapper getIdMapper() {
         return idMapper;
     }
 
@@ -134,7 +134,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @param idMapper ID mapper.
      */
-    public void setIdMapper(PortableIdMapper idMapper) {
+    public void setIdMapper(IgniteObjectIdMapper idMapper) {
         this.idMapper = idMapper;
     }
 
@@ -143,7 +143,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @return Serializer.
      */
-    public PortableSerializer getSerializer() {
+    public IgniteObjectSerializer getSerializer() {
         return serializer;
     }
 
@@ -152,7 +152,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @param serializer Serializer.
      */
-    public void setSerializer(PortableSerializer serializer) {
+    public void setSerializer(IgniteObjectSerializer serializer) {
         this.serializer = serializer;
     }
 
@@ -161,7 +161,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @return Types configuration.
      */
-    public Collection<PortableTypeConfiguration> getTypeConfigurations() {
+    public Collection<IgniteObjectConfiguration> getTypeConfigurations() {
         return typeCfgs;
     }
 
@@ -170,7 +170,7 @@ public class PortableMarshaller extends AbstractMarshaller {
      *
      * @param typeCfgs Type configurations.
      */
-    public void setTypeConfigurations(Collection<PortableTypeConfiguration> typeCfgs) {
+    public void setTypeConfigurations(Collection<IgniteObjectConfiguration> typeCfgs) {
         this.typeCfgs = typeCfgs;
     }
 
@@ -198,7 +198,7 @@ public class PortableMarshaller extends AbstractMarshaller {
 
     /**
      * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for
-     * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method.
+     * some specific type, use {@link IgniteObjectConfiguration#setMetaDataEnabled(Boolean)} method.
      * <p>
      * Default value if {@code true}.
      *
@@ -216,11 +216,11 @@ public class PortableMarshaller extends AbstractMarshaller {
     }
 
     /**
-     * If {@code true}, {@link PortableObject} will cache deserialized instance after
-     * {@link PortableObject#deserialize()} is called. All consequent calls of this
-     * method on the same instance of {@link PortableObject} will return that cached
+     * If {@code true}, {@link IgniteObject} will cache deserialized instance after
+     * {@link IgniteObject#deserialize()} is called. All consequent calls of this
+     * method on the same instance of {@link IgniteObject} will return that cached
      * value without actually deserializing portable object. If you need to override this
-     * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)}
+     * behaviour for some specific type, use {@link IgniteObjectConfiguration#setKeepDeserialized(Boolean)}
      * method.
      * <p>
      * Default value if {@code true}.
@@ -271,7 +271,7 @@ public class PortableMarshaller extends AbstractMarshaller {
             out.write(arr);
         }
         catch (IOException e) {
-            throw new PortableException("Failed to marshal the object: " + obj, e);
+            throw new IgniteObjectException("Failed to marshal the object: " + obj, e);
         }
     }
 
@@ -298,7 +298,7 @@ public class PortableMarshaller extends AbstractMarshaller {
             return impl.deserialize(buf.toByteArray(), clsLdr);
         }
         catch (IOException e) {
-            throw new PortableException("Failed to unmarshal the object from InputStream", e);
+            throw new IgniteObjectException("Failed to unmarshal the object from InputStream", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
deleted file mode 100644
index 377fcdc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableBuilder.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import org.apache.ignite.IgnitePortables;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable object builder. Provides ability to build portable objects dynamically without having class definitions.
- * <p>
- * Here is an example of how a portable object can be built dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder("org.project.MyObject");
- *
- * builder.setField("fieldA", "A");
- * builder.setField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- *
- * <p>
- * Also builder can be initialized by existing portable object. This allows changing some fields without affecting
- * other fields.
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder(person);
- *
- * builder.setField("name", "John");
- *
- * person = builder.build();
- * </pre>
- * </p>
- *
- * If you need to modify nested portable object you can get builder for nested object using
- * {@link #getField(String)}, changes made on nested builder will affect parent object,
- * for example:
- *
- * <pre name=code class=java>
- * PortableBuilder personBuilder = grid.portables().createBuilder(personPortableObj);
- * PortableBuilder addressBuilder = personBuilder.setField("address");
- *
- * addressBuilder.setField("city", "New York");
- *
- * personPortableObj = personBuilder.build();
- *
- * // Should be "New York".
- * String city = personPortableObj.getField("address").getField("city");
- * </pre>
- *
- * @see IgnitePortables#builder(int)
- * @see IgnitePortables#builder(String)
- * @see IgnitePortables#builder(PortableObject)
- */
-public interface PortableBuilder {
-    /**
-     * Returns value assigned to the specified field.
-     * If the value is a portable object instance of {@code GridPortableBuilder} will be returned,
-     * which can be modified.
-     * <p>
-     * Collections and maps returned from this method are modifiable.
-     *
-     * @param name Field name.
-     * @return Filed value.
-     */
-    public <T> T getField(String name);
-
-    /**
-     * Sets field value.
-     *
-     * @param name Field name.
-     * @param val Field value (cannot be {@code null}).
-     * @see PortableObject#metaData()
-     */
-    public PortableBuilder setField(String name, Object val);
-
-    /**
-     * Sets field value with value type specification.
-     * <p>
-     * Field type is needed for proper metadata update.
-     *
-     * @param name Field name.
-     * @param val Field value.
-     * @param type Field type.
-     * @see PortableObject#metaData()
-     */
-    public <T> PortableBuilder setField(String name, @Nullable T val, Class<? super T> type);
-
-    /**
-     * Sets field value.
-     * <p>
-     * This method should be used if field is portable object.
-     *
-     * @param name Field name.
-     * @param builder Builder for object field.
-     */
-    public PortableBuilder setField(String name, @Nullable PortableBuilder builder);
-
-    /**
-     * Removes field from this builder.
-     *
-     * @param fieldName Field name.
-     * @return {@code this} instance for chaining.
-     */
-    public PortableBuilder removeField(String fieldName);
-
-    /**
-     * Sets hash code for resulting portable object returned by {@link #build()} method.
-     * <p>
-     * If not set {@code 0} is used.
-     *
-     * @param hashCode Hash code.
-     * @return {@code this} instance for chaining.
-     */
-    public PortableBuilder hashCode(int hashCode);
-
-    /**
-     * Builds portable object.
-     *
-     * @return Portable object.
-     * @throws PortableException In case of error.
-     */
-    public PortableObject build() throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java
deleted file mode 100644
index 0f8d78b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableException.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import org.apache.ignite.IgniteException;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Exception indicating portable object serialization error.
- */
-public class PortableException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates portable exception with error message.
-     *
-     * @param msg Error message.
-     */
-    public PortableException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates portable exception with {@link Throwable} as a cause.
-     *
-     * @param cause Cause.
-     */
-    public PortableException(Throwable cause) {
-        super(cause);
-    }
-
-    /**
-     * Creates portable exception with error message and {@link Throwable} as a cause.
-     *
-     * @param msg Error message.
-     * @param cause Cause.
-     */
-    public PortableException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
deleted file mode 100644
index 368e415..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableIdMapper.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-
-/**
- * Type and field ID mapper for portable objects. Ignite never writes full
- * strings for field or type names. Instead, for performance reasons, Ignite
- * writes integer hash codes for type and field names. It has been tested that
- * hash code conflicts for the type names or the field names
- * within the same type are virtually non-existent and, to gain performance, it is safe
- * to work with hash codes. For the cases when hash codes for different types or fields
- * actually do collide {@code PortableIdMapper} allows to override the automatically
- * generated hash code IDs for the type and field names.
- * <p>
- * Portable ID mapper can be configured for all portable objects via {@link PortableMarshaller#getIdMapper()} method,
- * or for a specific portable type via {@link PortableTypeConfiguration#getIdMapper()} method.
- */
-public interface PortableIdMapper {
-    /**
-     * Gets type ID for provided class name.
-     * <p>
-     * If {@code 0} is returned, hash code of class simple name will be used.
-     *
-     * @param clsName Class name.
-     * @return Type ID.
-     */
-    public int typeId(String clsName);
-
-    /**
-     * Gets ID for provided field.
-     * <p>
-     * If {@code 0} is returned, hash code of field name will be used.
-     *
-     * @param typeId Type ID.
-     * @param fieldName Field name.
-     * @return Field ID.
-     */
-    public int fieldId(int typeId, String fieldName);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
deleted file mode 100644
index 0098ec3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableInvalidClassException.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Exception indicating that class needed for deserialization of portable object does not exist.
- * <p>
- * Thrown from {@link PortableObject#deserialize()} method.
- */
-public class PortableInvalidClassException extends PortableException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates invalid class exception with error message.
-     *
-     * @param msg Error message.
-     */
-    public PortableInvalidClassException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates invalid class exception with {@link Throwable} as a cause.
-     *
-     * @param cause Cause.
-     */
-    public PortableInvalidClassException(Throwable cause) {
-        super(cause);
-    }
-
-    /**
-     * Creates invalid class exception with error message and {@link Throwable} as a cause.
-     *
-     * @param msg Error message.
-     * @param cause Cause.
-     */
-    public PortableInvalidClassException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
deleted file mode 100644
index 4270885..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableMarshalAware.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-/**
- * Interface that allows to implement custom serialization
- * logic for portable objects. Portable objects are not required
- * to implement this interface, in which case Ignite will automatically
- * serialize portable objects using reflection.
- * <p>
- * This interface, in a way, is analogous to {@link java.io.Externalizable}
- * interface, which allows users to override default serialization logic,
- * usually for performance reasons. The only difference here is that portable
- * serialization is already very fast and implementing custom serialization
- * logic for portables does not provide significant performance gains.
- */
-public interface PortableMarshalAware {
-    /**
-     * Writes fields to provided writer.
-     *
-     * @param writer Portable object writer.
-     * @throws PortableException In case of error.
-     */
-    public void writePortable(PortableWriter writer) throws PortableException;
-
-    /**
-     * Reads fields from provided reader.
-     *
-     * @param reader Portable object reader.
-     * @throws PortableException In case of error.
-     */
-    public void readPortable(PortableReader reader) throws PortableException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/35b6d61f/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java b/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
deleted file mode 100644
index 4ea808b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/portable/PortableMetadata.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.portable;
-
-import java.util.Collection;
-import org.apache.ignite.IgnitePortables;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Portable type meta data. Metadata for portable types can be accessed from any of the
- * {@link IgnitePortables#metadata(String)} methods.
- * Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- */
-public interface PortableMetadata {
-    /**
-     * Gets portable type name.
-     *
-     * @return Portable type name.
-     */
-    public String typeName();
-
-    /**
-     * Gets collection of all field names for this portable type.
-     *
-     * @return Collection of all field names for this portable type.
-     */
-    public Collection<String> fields();
-
-    /**
-     * Gets name of the field type for a given field.
-     *
-     * @param fieldName Field name.
-     * @return Field type name.
-     */
-    @Nullable public String fieldTypeName(String fieldName);
-
-    /**
-     * Portable objects can optionally specify custom key-affinity mapping in the
-     * configuration. This method returns the name of the field which should be
-     * used for the key-affinity mapping.
-     *
-     * @return Affinity key field name.
-     */
-    @Nullable public String affinityKeyFieldName();
-}
\ No newline at end of file