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

[33/44] incubator-ignite git commit: ignite-1258: renaming portalbe internal classes

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectArrayLazyValue.java
deleted file mode 100644
index 5557e4b..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectArrayLazyValue.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.portable.*;
-
-/**
- *
- */
-class GridPortableObjectArrayLazyValue extends GridPortableAbstractLazyValue {
-    /** */
-    private Object[] lazyValsArr;
-
-    /** */
-    private int compTypeId;
-
-    /** */
-    private String clsName;
-
-    /**
-     * @param reader Reader.
-     */
-    protected GridPortableObjectArrayLazyValue(GridPortableBuilderReader reader) {
-        super(reader, reader.position() - 1);
-
-        int typeId = reader.readInt();
-
-        if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID) {
-            clsName = reader.readString();
-
-            Class cls;
-
-            try {
-                // TODO: GG-10396 - Is class loader needed here?
-                cls = U.forName(reader.readString(), null);
-            }
-            catch (ClassNotFoundException e) {
-                throw new PortableInvalidClassException("Failed to load the class: " + clsName, e);
-            }
-
-            compTypeId = reader.portableContext().descriptorForClass(cls).typeId();
-        }
-        else {
-            compTypeId = typeId;
-            clsName = null;
-        }
-
-        int size = reader.readInt();
-
-        lazyValsArr = new Object[size];
-
-        for (int i = 0; i < size; i++)
-            lazyValsArr[i] = reader.parseValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        for (int i = 0; i < lazyValsArr.length; i++) {
-            if (lazyValsArr[i] instanceof GridPortableLazyValue)
-                lazyValsArr[i] = ((GridPortableLazyValue)lazyValsArr[i]).value();
-        }
-
-        return lazyValsArr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(GridPortableWriterExImpl writer, GridPortableBuilderSerializer ctx) {
-        if (clsName == null)
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, compTypeId);
-        else
-            ctx.writeArray(writer, GridPortableMarshaller.OBJ_ARR, lazyValsArr, clsName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectEx.java
deleted file mode 100644
index 379b1b9..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectEx.java
+++ /dev/null
@@ -1,213 +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.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.portable.*;
-
-import org.jetbrains.annotations.*;
-
-import java.math.*;
-import java.util.*;
-
-/**
- * Internal portable object interface.
- */
-public abstract class GridPortableObjectEx 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(GridPortableReaderContext 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 GridPortableObjectEx))
-            return false;
-
-        GridPortableObjectEx otherPo = (GridPortableObjectEx)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(GridPortableReaderContext 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 GridPortableObjectEx) {
-                        GridPortableObjectEx po = (GridPortableObjectEx)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 {
-            GridPortableReaderContext ctx = new GridPortableReaderContext();
-
-            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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectImpl.java
deleted file mode 100644
index 773c69d..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectImpl.java
+++ /dev/null
@@ -1,383 +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.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.portable.streams.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.portable.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.extensions.communication.*;
-import org.apache.ignite.portable.*;
-
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.nio.*;
-
-/**
- * Portable object implementation.
- */
-@IgniteCodeGeneratingFail // Fields arr and start should not be generated by MessageCodeGenerator.
-public final class GridPortableObjectImpl extends GridPortableObjectEx implements Externalizable,
-    Message, CacheObject, KeyCacheObject {
-    /** */
-    public static final byte TYPE_PORTABLE = 100;
-
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final GridPortablePrimitives PRIM = GridPortablePrimitives.get();
-
-    /** */
-    @GridDirectTransient
-    private GridPortableContext ctx;
-
-    /** */
-    private byte[] arr;
-
-    /** */
-    private int start;
-
-    /** */
-    @GridDirectTransient
-    private Object obj;
-
-    /** */
-    @GridDirectTransient
-    private boolean detachAllowed;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public GridPortableObjectImpl() {
-        // No-op.
-    }
-
-    /**
-     * @param ctx Context.
-     * @param arr Array.
-     * @param start Start.
-     */
-    public GridPortableObjectImpl(GridPortableContext 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 internal() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@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 (GridPortableObjectImpl)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 GridPortableObjectImpl(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 GridPortableContext context() {
-        return ctx;
-    }
-
-    /**
-     * @param ctx Context.
-     */
-    public void context(GridPortableContext 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 + 2);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public PortableMetadata metaData() throws PortableException {
-        if (ctx == null)
-            throw new PortableException("GridPortableContext is not set for the object.");
-
-        return ctx.metaData(typeId());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <F> F field(String fieldName) throws PortableException {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx, arr, start, null);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(GridPortableReaderContext rCtx, String fieldName) {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx,
-            new GridPortableHeapInputStream(arr),
-            start,
-            null,
-            rCtx);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasField(String fieldName) {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(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) {
-            // TODO: GG-10396 - Deserialize with proper class loader.
-            GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx, arr, start, null);
-
-            obj0 = reader.deserialize();
-
-            GridPortableClassDescriptor desc = reader.descriptor();
-
-            assert desc != null;
-
-            if (desc.keepDeserialized())
-                obj = obj0;
-        }
-
-        return (T)obj0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public PortableObject clone() throws CloneNotSupportedException {
-        return super.clone();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return PRIM.readInt(arr, start + 6);
-    }
-
-    /** {@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 = (GridPortableContext)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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectOffheapImpl.java
deleted file mode 100644
index ba526d9..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableObjectOffheapImpl.java
+++ /dev/null
@@ -1,238 +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.*;
-import org.apache.ignite.internal.portable.streams.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.extensions.communication.*;
-import org.apache.ignite.portable.*;
-
-import org.jetbrains.annotations.*;
-import sun.misc.*;
-
-import java.io.*;
-import java.nio.*;
-
-/**
- *  Portable object implementation over offheap memory
- */
-public class GridPortableObjectOffheapImpl extends GridPortableObjectEx implements Externalizable, CacheObject {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private final GridPortableContext ctx;
-
-    /** */
-    private final long ptr;
-
-    /** */
-    private final int start;
-
-    /** */
-    private final int size;
-
-    /**
-     * For {@link Externalizable} (not supported).
-     */
-    public GridPortableObjectOffheapImpl() {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @param ctx Context.
-     * @param ptr Memory address.
-     * @param start Object start.
-     * @param size Memory size.
-     */
-    public GridPortableObjectOffheapImpl(GridPortableContext 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 GridPortableObjectImpl(ctx, U.copyMemory(ptr, size), start);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int typeId() {
-        return UNSAFE.getInt(ptr + start + 2);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int length() {
-        return UNSAFE.getInt(ptr + start + GridPortableMarshaller.TOTAL_LEN_POS);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return UNSAFE.getInt(ptr + start + 6);
-    }
-
-    /** {@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("GridPortableContext is not set for the object.");
-
-        return ctx.metaData(typeId());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <F> F field(String fieldName) throws PortableException {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx,
-            new GridPortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override protected <F> F field(GridPortableReaderContext rCtx, String fieldName) {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx,
-            new GridPortableOffheapInputStream(ptr, size, false),
-            start,
-            null,
-            rCtx);
-
-        return (F)reader.unmarshal(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasField(String fieldName) {
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(ctx,
-            new GridPortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return reader.hasField(fieldName);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T deserialize() throws PortableException {
-        // TODO: GG-10396 - Deserialize with proper class loader.
-        GridGridPortableReaderExImpl reader = new GridGridPortableReaderExImpl(
-            ctx,
-            new GridPortableOffheapInputStream(ptr, size, false),
-            start,
-            null);
-
-        return (T)reader.deserialize();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("CloneDoesntCallSuperClone")
-    @Override public PortableObject clone() throws CloneNotSupportedException {
-        return heapCopy();
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte type() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@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.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainLazyValue.java
deleted file mode 100644
index c3ebcd4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainLazyValue.java
+++ /dev/null
@@ -1,47 +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;
-
-/**
- *
- */
-class GridPortablePlainLazyValue extends GridPortableAbstractLazyValue {
-    /** */
-    protected final int len;
-
-    /**
-     * @param reader Reader
-     * @param valOff Offset
-     * @param len Length.
-     */
-    protected GridPortablePlainLazyValue(GridPortableBuilderReader reader, int valOff, int len) {
-        super(reader, valOff);
-
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Object init() {
-        return reader.reader().unmarshal(valOff);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(GridPortableWriterExImpl writer, GridPortableBuilderSerializer ctx) {
-        writer.write(reader.array(), valOff, len);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainPortableObject.java
deleted file mode 100644
index 1e8e6f3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePlainPortableObject.java
+++ /dev/null
@@ -1,50 +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.*;
-
-/**
- *
- */
-public class GridPortablePlainPortableObject implements GridPortableLazyValue {
-    /** */
-    private final PortableObject portableObj;
-
-    /**
-     * @param portableObj Portable object.
-     */
-    public GridPortablePlainPortableObject(PortableObject portableObj) {
-        this.portableObj = portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        return portableObj;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(GridPortableWriterExImpl writer, GridPortableBuilderSerializer ctx) {
-        PortableObject val = portableObj;
-
-        if (val instanceof GridPortableObjectOffheapImpl)
-            val = ((GridPortableObjectOffheapImpl)val).heapCopy();
-
-        writer.doWritePortableObject((GridPortableObjectImpl)val);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePrimitives.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePrimitives.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePrimitives.java
deleted file mode 100644
index eca91c0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortablePrimitives.java
+++ /dev/null
@@ -1,773 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.util.*;
-
-import sun.misc.*;
-
-import static java.nio.ByteOrder.*;
-
-/**
- * Primitives writer.
- */
-abstract class GridPortablePrimitives {
-    /** */
-    private static final GridPortablePrimitives INSTANCE =
-        nativeOrder() == LITTLE_ENDIAN ? new UnsafePrimitives() : new BytePrimitives();
-
-    /**
-     * @return Primitives writer.
-     */
-    static GridPortablePrimitives get() {
-        return INSTANCE;
-    }
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeByte(byte[] arr, int off, byte val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract byte readByte(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeShort(byte[] arr, int off, short val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract short readShort(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeInt(byte[] arr, int off, int val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract int readInt(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeLong(byte[] arr, int off, long val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract long readLong(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeFloat(byte[] arr, int off, float val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract float readFloat(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeDouble(byte[] arr, int off, double val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract double readDouble(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeChar(byte[] arr, int off, char val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract char readChar(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeBoolean(byte[] arr, int off, boolean val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract boolean readBoolean(byte[] arr, int off);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeByteArray(byte[] arr, int off, byte[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract byte[] readByteArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeShortArray(byte[] arr, int off, short[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract short[] readShortArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeIntArray(byte[] arr, int off, int[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract int[] readIntArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeLongArray(byte[] arr, int off, long[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract long[] readLongArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeFloatArray(byte[] arr, int off, float[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract float[] readFloatArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeDoubleArray(byte[] arr, int off, double[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract double[] readDoubleArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeCharArray(byte[] arr, int off, char[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract char[] readCharArray(byte[] arr, int off, int len);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @param val Value.
-     */
-    abstract void writeBooleanArray(byte[] arr, int off, boolean[] val);
-
-    /**
-     * @param arr Array.
-     * @param off Offset.
-     * @return Value.
-     */
-    abstract boolean[] readBooleanArray(byte[] arr, int off, int len);
-
-    /** */
-    private static class UnsafePrimitives extends GridPortablePrimitives {
-        /** */
-        private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-        /** */
-        private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-        /** */
-        private static final long SHORT_ARR_OFF = UNSAFE.arrayBaseOffset(short[].class);
-
-        /** */
-        private static final long INT_ARR_OFF = UNSAFE.arrayBaseOffset(int[].class);
-
-        /** */
-        private static final long LONG_ARR_OFF = UNSAFE.arrayBaseOffset(long[].class);
-
-        /** */
-        private static final long FLOAT_ARR_OFF = UNSAFE.arrayBaseOffset(float[].class);
-
-        /** */
-        private static final long DOUBLE_ARR_OFF = UNSAFE.arrayBaseOffset(double[].class);
-
-        /** */
-        private static final long CHAR_ARR_OFF = UNSAFE.arrayBaseOffset(char[].class);
-
-        /** */
-        private static final long BOOLEAN_ARR_OFF = UNSAFE.arrayBaseOffset(boolean[].class);
-
-        /** {@inheritDoc} */
-        @Override void writeByte(byte[] arr, int off, byte val) {
-            UNSAFE.putByte(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override byte readByte(byte[] arr, int off) {
-            return UNSAFE.getByte(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeShort(byte[] arr, int off, short val) {
-            UNSAFE.putShort(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override short readShort(byte[] arr, int off) {
-            return UNSAFE.getShort(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeInt(byte[] arr, int off, int val) {
-            UNSAFE.putInt(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override int readInt(byte[] arr, int off) {
-            return UNSAFE.getInt(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeLong(byte[] arr, int off, long val) {
-            UNSAFE.putLong(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override long readLong(byte[] arr, int off) {
-            return UNSAFE.getLong(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeFloat(byte[] arr, int off, float val) {
-            UNSAFE.putFloat(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override float readFloat(byte[] arr, int off) {
-            return UNSAFE.getFloat(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeDouble(byte[] arr, int off, double val) {
-            UNSAFE.putDouble(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override double readDouble(byte[] arr, int off) {
-            return UNSAFE.getDouble(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeChar(byte[] arr, int off, char val) {
-            UNSAFE.putChar(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override char readChar(byte[] arr, int off) {
-            return UNSAFE.getChar(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeBoolean(byte[] arr, int off, boolean val) {
-            UNSAFE.putBoolean(arr, BYTE_ARR_OFF + off, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean readBoolean(byte[] arr, int off) {
-            return UNSAFE.getBoolean(arr, BYTE_ARR_OFF + off);
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeByteArray(byte[] arr, int off, byte[] val) {
-            UNSAFE.copyMemory(val, BYTE_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override byte[] readByteArray(byte[] arr, int off, int len) {
-            byte[] arr0 = new byte[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, BYTE_ARR_OFF, len);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeShortArray(byte[] arr, int off, short[] val) {
-            UNSAFE.copyMemory(val, SHORT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 1);
-        }
-
-        /** {@inheritDoc} */
-        @Override short[] readShortArray(byte[] arr, int off, int len) {
-            short[] arr0 = new short[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, SHORT_ARR_OFF, len << 1);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeIntArray(byte[] arr, int off, int[] val) {
-            UNSAFE.copyMemory(val, INT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 2);
-        }
-
-        /** {@inheritDoc} */
-        @Override int[] readIntArray(byte[] arr, int off, int len) {
-            int[] arr0 = new int[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, INT_ARR_OFF, len << 2);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeLongArray(byte[] arr, int off, long[] val) {
-            UNSAFE.copyMemory(val, LONG_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 3);
-        }
-
-        /** {@inheritDoc} */
-        @Override long[] readLongArray(byte[] arr, int off, int len) {
-            long[] arr0 = new long[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, LONG_ARR_OFF, len << 3);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeFloatArray(byte[] arr, int off, float[] val) {
-            UNSAFE.copyMemory(val, FLOAT_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 2);
-        }
-
-        /** {@inheritDoc} */
-        @Override float[] readFloatArray(byte[] arr, int off, int len) {
-            float[] arr0 = new float[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, FLOAT_ARR_OFF, len << 2);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeDoubleArray(byte[] arr, int off, double[] val) {
-            UNSAFE.copyMemory(val, DOUBLE_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 3);
-        }
-
-        /** {@inheritDoc} */
-        @Override double[] readDoubleArray(byte[] arr, int off, int len) {
-            double[] arr0 = new double[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, DOUBLE_ARR_OFF, len << 3);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeCharArray(byte[] arr, int off, char[] val) {
-            UNSAFE.copyMemory(val, CHAR_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length << 1);
-        }
-
-        /** {@inheritDoc} */
-        @Override char[] readCharArray(byte[] arr, int off, int len) {
-            char[] arr0 = new char[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, CHAR_ARR_OFF, len << 1);
-
-            return arr0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeBooleanArray(byte[] arr, int off, boolean[] val) {
-            UNSAFE.copyMemory(val, BOOLEAN_ARR_OFF, arr, BYTE_ARR_OFF + off, val.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean[] readBooleanArray(byte[] arr, int off, int len) {
-            boolean[] arr0 = new boolean[len];
-
-            UNSAFE.copyMemory(arr, BYTE_ARR_OFF + off, arr0, BOOLEAN_ARR_OFF, len);
-
-            return arr0;
-        }
-    }
-
-    /** */
-    private static class BytePrimitives extends GridPortablePrimitives {
-        /** {@inheritDoc} */
-        @Override void writeByte(byte[] arr, int off, byte val) {
-            arr[off] = val;
-        }
-
-        /** {@inheritDoc} */
-        @Override byte readByte(byte[] arr, int off) {
-            return arr[off];
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeShort(byte[] arr, int off, short val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off] = (byte)((val >>> 8) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override short readShort(byte[] arr, int off) {
-            short val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off] & 0xff) << 8;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeInt(byte[] arr, int off, int val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off++] = (byte)((val >>> 8) & 0xff);
-            arr[off++] = (byte)((val >>> 16) & 0xff);
-            arr[off] = (byte)((val >>> 24) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override int readInt(byte[] arr, int off) {
-            int val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off++] & 0xff) << 8;
-            val |= (arr[off++] & 0xff) << 16;
-            val |= (arr[off] & 0xff) << 24;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeLong(byte[] arr, int off, long val) {
-            arr[off++] = (byte)(val & 0xffL);
-            arr[off++] = (byte)((val >>> 8) & 0xffL);
-            arr[off++] = (byte)((val >>> 16) & 0xffL);
-            arr[off++] = (byte)((val >>> 24) & 0xffL);
-            arr[off++] = (byte)((val >>> 32) & 0xffL);
-            arr[off++] = (byte)((val >>> 40) & 0xffL);
-            arr[off++] = (byte)((val >>> 48) & 0xffL);
-            arr[off] = (byte)((val >>> 56) & 0xffL);
-        }
-
-        /** {@inheritDoc} */
-        @Override long readLong(byte[] arr, int off) {
-            long val = 0;
-
-            val |= (arr[off++] & 0xffL);
-            val |= (arr[off++] & 0xffL) << 8;
-            val |= (arr[off++] & 0xffL) << 16;
-            val |= (arr[off++] & 0xffL) << 24;
-            val |= (arr[off++] & 0xffL) << 32;
-            val |= (arr[off++] & 0xffL) << 40;
-            val |= (arr[off++] & 0xffL) << 48;
-            val |= (arr[off] & 0xffL) << 56;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeFloat(byte[] arr, int off, float val) {
-            writeInt(arr, off, Float.floatToIntBits(val));
-        }
-
-        /** {@inheritDoc} */
-        @Override float readFloat(byte[] arr, int off) {
-            return Float.intBitsToFloat(readInt(arr, off));
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeDouble(byte[] arr, int off, double val) {
-            writeLong(arr, off, Double.doubleToLongBits(val));
-        }
-
-        /** {@inheritDoc} */
-        @Override double readDouble(byte[] arr, int off) {
-            return Double.longBitsToDouble(readLong(arr, off));
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeChar(byte[] arr, int off, char val) {
-            arr[off++] = (byte)(val & 0xff);
-            arr[off] = (byte)((val >>> 8) & 0xff);
-        }
-
-        /** {@inheritDoc} */
-        @Override char readChar(byte[] arr, int off) {
-            char val = 0;
-
-            val |= (arr[off++] & 0xff);
-            val |= (arr[off] & 0xff) << 8;
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeBoolean(byte[] arr, int off, boolean val) {
-            arr[off] = (byte)(val ? 1 : 0);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean readBoolean(byte[] arr, int off) {
-            return arr[off] != 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeByteArray(byte[] arr, int off, byte[] val) {
-            for (byte b : val)
-                arr[off++] = b;
-        }
-
-        /** {@inheritDoc} */
-        @Override byte[] readByteArray(byte[] arr, int off, int len) {
-            byte[] val = new byte[len];
-
-            for (int i = 0; i < len; i++)
-                val[i] = arr[off++];
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeShortArray(byte[] arr, int off, short[] val) {
-            for (short s : val) {
-                writeShort(arr, off, s);
-
-                off += 2;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override short[] readShortArray(byte[] arr, int off, int len) {
-            short[] val = new short[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readShort(arr, off);
-
-                off += 2;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeIntArray(byte[] arr, int off, int[] val) {
-            for (int i : val) {
-                writeInt(arr, off, i);
-
-                off += 4;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override int[] readIntArray(byte[] arr, int off, int len) {
-            int[] val = new int[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readInt(arr, off);
-
-                off += 4;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeLongArray(byte[] arr, int off, long[] val) {
-            for (long l : val) {
-                writeLong(arr, off, l);
-
-                off += 8;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override long[] readLongArray(byte[] arr, int off, int len) {
-            long[] val = new long[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readLong(arr, off);
-
-                off += 8;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeFloatArray(byte[] arr, int off, float[] val) {
-            for (float f : val) {
-                writeFloat(arr, off, f);
-
-                off += 4;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override float[] readFloatArray(byte[] arr, int off, int len) {
-            float[] val = new float[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readFloat(arr, off);
-
-                off += 4;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeDoubleArray(byte[] arr, int off, double[] val) {
-            for (double d : val) {
-                writeDouble(arr, off, d);
-
-                off += 8;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override double[] readDoubleArray(byte[] arr, int off, int len) {
-            double[] val = new double[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readDouble(arr, off);
-
-                off += 8;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeCharArray(byte[] arr, int off, char[] val) {
-            for (char c : val) {
-                writeChar(arr, off, c);
-
-                off += 2;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override char[] readCharArray(byte[] arr, int off, int len) {
-            char[] val = new char[len];
-
-            for (int i = 0; i < len; i++) {
-                val[i] = readChar(arr, off);
-
-                off += 2;
-            }
-
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override void writeBooleanArray(byte[] arr, int off, boolean[] val) {
-            for (boolean b : val)
-                writeBoolean(arr, off++, b);
-        }
-
-        /** {@inheritDoc} */
-        @Override boolean[] readBooleanArray(byte[] arr, int off, int len) {
-            boolean[] val = new boolean[len];
-
-            for (int i = 0; i < len; i++)
-                val[i] = readBoolean(arr, off++);
-
-            return val;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawReaderEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawReaderEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawReaderEx.java
deleted file mode 100644
index c01fd81..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawReaderEx.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.*;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Extended reader interface.
- */
-public interface GridPortableRawReaderEx extends PortableRawReader {
-    /**
-     * @return Object.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public Object readObjectDetached() throws PortableException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawWriterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawWriterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawWriterEx.java
deleted file mode 100644
index c0b2f9f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableRawWriterEx.java
+++ /dev/null
@@ -1,44 +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.processors.portable.*;
-import org.apache.ignite.portable.*;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Extended writer interface.
- */
-public interface GridPortableRawWriterEx 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 GridPortableOutputStream out();
-
-    /**
-     * Cleans resources.
-     */
-    @Override public void close();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableReaderContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableReaderContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableReaderContext.java
deleted file mode 100644
index 58281d4..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableReaderContext.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.portable;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.portable.*;
-
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Reader context.
- */
-class GridPortableReaderContext {
-    /** */
-    private Map<Integer, Object> oHandles;
-
-    /** */
-    private Map<Integer, PortableObject> poHandles;
-
-    /**
-     * @param handle Handle.
-     * @param obj Object.
-     */
-    void setObjectHandler(int handle, Object obj) {
-        assert obj != null;
-
-        if (oHandles == null)
-            oHandles = new HashMap<>(3, 1.0f);
-
-        oHandles.put(handle, obj);
-    }
-
-    /**
-     * @param handle Handle.
-     * @param po Portable object.
-     */
-    void setPortableHandler(int handle, PortableObject po) {
-        assert po != null;
-
-        if (poHandles == null)
-            poHandles = new HashMap<>(3, 1.0f);
-
-        poHandles.put(handle, po);
-    }
-
-    /**
-     * @param handle Handle.
-     * @return Object.
-     */
-    @Nullable Object getObjectByHandle(int handle) {
-        return oHandles != null ? oHandles.get(handle) : null;
-    }
-
-    /**
-     * @param handle Handle.
-     * @return Object.
-     */
-    @Nullable PortableObject getPortableByHandle(int handle) {
-        return poHandles != null ? poHandles.get(handle) : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridPortableReaderContext.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableThreadLocalMemoryAllocator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableThreadLocalMemoryAllocator.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableThreadLocalMemoryAllocator.java
deleted file mode 100644
index 29d4fb6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableThreadLocalMemoryAllocator.java
+++ /dev/null
@@ -1,163 +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.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import sun.misc.*;
-
-import static org.apache.ignite.IgniteSystemProperties.*;
-
-/**
- * Thread-local memory allocator.
- */
-public class GridPortableThreadLocalMemoryAllocator implements GridPortableMemoryAllocator {
-    /** Memory allocator instance. */
-    public static final GridPortableThreadLocalMemoryAllocator THREAD_LOCAL_ALLOC =
-        new GridPortableThreadLocalMemoryAllocator();
-
-    /** Holders. */
-    private static final ThreadLocal<ByteArrayHolder> holders = new ThreadLocal<>();
-
-    /** Unsafe instance. */
-    protected static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** Array offset: byte. */
-    protected static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /**
-     * Ensures singleton.
-     */
-    private GridPortableThreadLocalMemoryAllocator() {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] allocate(int size) {
-        ByteArrayHolder holder = holders.get();
-
-        if (holder == null)
-            holders.set(holder = new ByteArrayHolder());
-
-        if (holder.acquired)
-            return new byte[size];
-
-        holder.acquired = true;
-
-        if (holder.data == null || size > holder.data.length)
-            holder.data = new byte[size];
-
-        return holder.data;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] reallocate(byte[] data, int size) {
-        ByteArrayHolder holder = holders.get();
-
-        assert holder != null;
-
-        byte[] newData = new byte[size];
-
-        if (holder.data == data)
-            holder.data = newData;
-
-        UNSAFE.copyMemory(data, BYTE_ARR_OFF, newData, BYTE_ARR_OFF, data.length);
-
-        return newData;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void release(byte[] data, int maxMsgSize) {
-        ByteArrayHolder holder = holders.get();
-
-        assert holder != null;
-
-        if (holder.data != data)
-            return;
-
-        holder.maxMsgSize = maxMsgSize;
-        holder.acquired = false;
-
-        holder.shrink();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long allocateDirect(int size) {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long reallocateDirect(long addr, int size) {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void releaseDirect(long addr) {
-        // No-op
-    }
-
-    /**
-     * Checks whether a thread-local array is acquired or not.
-     * The function is used by Unit tests.
-     *
-     * @return {@code true} if acquired {@code false} otherwise.
-     */
-    public boolean isThreadLocalArrayAcquired() {
-        ByteArrayHolder holder = holders.get();
-
-        return holder != null && holder.acquired;
-    }
-
-    /**
-     * Thread-local byte array holder.
-     */
-    private static class ByteArrayHolder {
-        /** */
-        private static final Long CHECK_FREQ = Long.getLong(IGNITE_MARSHAL_BUFFERS_RECHECK, 10000);
-
-        /** Data array */
-        private byte[] data;
-
-        /** Max message size detected between checks. */
-        private int maxMsgSize;
-
-        /** Last time array size is checked. */
-        private long lastCheck = U.currentTimeMillis();
-
-        /** Whether the holder is acquired or not. */
-        private boolean acquired;
-
-        /**
-         * Shrinks array size if needed.
-         */
-        private void shrink() {
-            long now = U.currentTimeMillis();
-
-            if (now - lastCheck >= CHECK_FREQ) {
-                int halfSize = data.length >> 1;
-
-                if (maxMsgSize < halfSize)
-                    data = new byte[halfSize];
-
-                lastCheck = now;
-            }
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableUtils.java
deleted file mode 100644
index cb4d246..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableUtils.java
+++ /dev/null
@@ -1,380 +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.*;
-
-import org.jetbrains.annotations.*;
-
-import java.math.*;
-import java.sql.*;
-import java.util.*;
-import java.util.Date;
-
-import static org.apache.ignite.internal.portable.GridPortableMarshaller.*;
-
-/**
- *
- */
-public class GridPortableUtils {
-    /** */
-    public static final Map<Class<?>, Byte> PLAIN_CLASS_TO_FLAG = new HashMap<>();
-
-    /** */
-    public static final Map<Byte, Class<?>> FLAG_TO_CLASS = new HashMap<>();
-
-    /** {@code true} if serialized value of this type cannot contain references to objects. */
-    private static final boolean[] PLAIN_TYPE_FLAG = new boolean[102];
-
-    /** Portable classes. */
-    private static final Collection<Class<?>> PORTABLE_CLS = new HashSet<>();
-
-    /**
-     *
-     */
-    static {
-        PORTABLE_CLS.add(Byte.class);
-        PORTABLE_CLS.add(Short.class);
-        PORTABLE_CLS.add(Integer.class);
-        PORTABLE_CLS.add(Long.class);
-        PORTABLE_CLS.add(Float.class);
-        PORTABLE_CLS.add(Double.class);
-        PORTABLE_CLS.add(Character.class);
-        PORTABLE_CLS.add(Boolean.class);
-        PORTABLE_CLS.add(String.class);
-        PORTABLE_CLS.add(UUID.class);
-        PORTABLE_CLS.add(Date.class);
-        PORTABLE_CLS.add(Timestamp.class);
-        PORTABLE_CLS.add(BigDecimal.class);
-        PORTABLE_CLS.add(byte[].class);
-        PORTABLE_CLS.add(short[].class);
-        PORTABLE_CLS.add(int[].class);
-        PORTABLE_CLS.add(long[].class);
-        PORTABLE_CLS.add(float[].class);
-        PORTABLE_CLS.add(double[].class);
-        PORTABLE_CLS.add(char[].class);
-        PORTABLE_CLS.add(boolean[].class);
-        PORTABLE_CLS.add(String[].class);
-        PORTABLE_CLS.add(UUID[].class);
-        PORTABLE_CLS.add(Date[].class);
-        PORTABLE_CLS.add(Timestamp[].class);
-        PORTABLE_CLS.add(BigDecimal[].class);
-    }
-
-    /**
-     *
-     */
-    static {
-        PLAIN_CLASS_TO_FLAG.put(Byte.class, GridPortableMarshaller.BYTE);
-        PLAIN_CLASS_TO_FLAG.put(Short.class, GridPortableMarshaller.SHORT);
-        PLAIN_CLASS_TO_FLAG.put(Integer.class, GridPortableMarshaller.INT);
-        PLAIN_CLASS_TO_FLAG.put(Long.class, GridPortableMarshaller.LONG);
-        PLAIN_CLASS_TO_FLAG.put(Float.class, GridPortableMarshaller.FLOAT);
-        PLAIN_CLASS_TO_FLAG.put(Double.class, GridPortableMarshaller.DOUBLE);
-        PLAIN_CLASS_TO_FLAG.put(Character.class, GridPortableMarshaller.CHAR);
-        PLAIN_CLASS_TO_FLAG.put(Boolean.class, GridPortableMarshaller.BOOLEAN);
-        PLAIN_CLASS_TO_FLAG.put(BigDecimal.class, GridPortableMarshaller.DECIMAL);
-        PLAIN_CLASS_TO_FLAG.put(String.class, GridPortableMarshaller.STRING);
-        PLAIN_CLASS_TO_FLAG.put(UUID.class, GridPortableMarshaller.UUID);
-        PLAIN_CLASS_TO_FLAG.put(Date.class, GridPortableMarshaller.DATE);
-
-        PLAIN_CLASS_TO_FLAG.put(byte[].class, GridPortableMarshaller.BYTE_ARR);
-        PLAIN_CLASS_TO_FLAG.put(short[].class, GridPortableMarshaller.SHORT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(int[].class, GridPortableMarshaller.INT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(long[].class, GridPortableMarshaller.LONG_ARR);
-        PLAIN_CLASS_TO_FLAG.put(float[].class, GridPortableMarshaller.FLOAT_ARR);
-        PLAIN_CLASS_TO_FLAG.put(double[].class, GridPortableMarshaller.DOUBLE_ARR);
-        PLAIN_CLASS_TO_FLAG.put(char[].class, GridPortableMarshaller.CHAR_ARR);
-        PLAIN_CLASS_TO_FLAG.put(boolean[].class, GridPortableMarshaller.BOOLEAN_ARR);
-        PLAIN_CLASS_TO_FLAG.put(BigDecimal[].class, GridPortableMarshaller.DECIMAL_ARR);
-        PLAIN_CLASS_TO_FLAG.put(String[].class, GridPortableMarshaller.STRING_ARR);
-        PLAIN_CLASS_TO_FLAG.put(UUID[].class, GridPortableMarshaller.UUID_ARR);
-        PLAIN_CLASS_TO_FLAG.put(Date[].class, GridPortableMarshaller.DATE_ARR);
-
-        for (Map.Entry<Class<?>, Byte> entry : PLAIN_CLASS_TO_FLAG.entrySet())
-            FLAG_TO_CLASS.put(entry.getValue(), entry.getKey());
-
-        PLAIN_CLASS_TO_FLAG.put(byte.class, GridPortableMarshaller.BYTE);
-        PLAIN_CLASS_TO_FLAG.put(short.class, GridPortableMarshaller.SHORT);
-        PLAIN_CLASS_TO_FLAG.put(int.class, GridPortableMarshaller.INT);
-        PLAIN_CLASS_TO_FLAG.put(long.class, GridPortableMarshaller.LONG);
-        PLAIN_CLASS_TO_FLAG.put(float.class, GridPortableMarshaller.FLOAT);
-        PLAIN_CLASS_TO_FLAG.put(double.class, GridPortableMarshaller.DOUBLE);
-        PLAIN_CLASS_TO_FLAG.put(char.class, GridPortableMarshaller.CHAR);
-        PLAIN_CLASS_TO_FLAG.put(boolean.class, GridPortableMarshaller.BOOLEAN);
-
-        for (byte b : new byte[] {
-            BYTE, SHORT, INT, LONG, FLOAT, DOUBLE,
-            CHAR, BOOLEAN, DECIMAL, STRING, UUID, DATE,
-            BYTE_ARR, SHORT_ARR, INT_ARR, LONG_ARR, FLOAT_ARR, DOUBLE_ARR,
-            CHAR_ARR, BOOLEAN_ARR, DECIMAL_ARR, STRING_ARR, UUID_ARR, DATE_ARR,
-            ENUM, ENUM_ARR, NULL}) {
-
-            PLAIN_TYPE_FLAG[b] = true;
-        }
-    }
-
-    /**
-     * Write value with flag. e.g. writePlainObject(writer, (byte)77) will write two byte: {BYTE, 77}.
-     *
-     * @param writer W
-     * @param val Value.
-     */
-    public static void writePlainObject(GridPortableWriterExImpl writer, Object val) {
-        Byte flag = PLAIN_CLASS_TO_FLAG.get(val.getClass());
-
-        if (flag == null)
-            throw new IllegalArgumentException("Can't write object with type: " + val.getClass());
-
-        switch (flag) {
-            case BYTE:
-                writer.writeByte(flag);
-                writer.writeByte((Byte)val);
-
-                break;
-
-            case SHORT:
-                writer.writeByte(flag);
-                writer.writeShort((Short)val);
-
-                break;
-
-            case INT:
-                writer.writeByte(flag);
-                writer.writeInt((Integer)val);
-
-                break;
-
-            case LONG:
-                writer.writeByte(flag);
-                writer.writeLong((Long)val);
-
-                break;
-
-            case FLOAT:
-                writer.writeByte(flag);
-                writer.writeFloat((Float)val);
-
-                break;
-
-            case DOUBLE:
-                writer.writeByte(flag);
-                writer.writeDouble((Double)val);
-
-                break;
-
-            case CHAR:
-                writer.writeByte(flag);
-                writer.writeChar((Character)val);
-
-                break;
-
-            case BOOLEAN:
-                writer.writeByte(flag);
-                writer.writeBoolean((Boolean)val);
-
-                break;
-
-            case DECIMAL:
-                writer.doWriteDecimal((BigDecimal)val);
-
-                break;
-
-            case STRING:
-                writer.doWriteString((String)val);
-
-                break;
-
-            case UUID:
-                writer.doWriteUuid((UUID)val);
-
-                break;
-
-            case DATE:
-                if (val instanceof Timestamp)
-                    writer.doWriteTimestamp((Timestamp)val);
-                else
-                    writer.doWriteDate((Date)val);
-
-                break;
-
-            case BYTE_ARR:
-                writer.doWriteByteArray((byte[])val);
-
-                break;
-
-            case SHORT_ARR:
-                writer.doWriteShortArray((short[])val);
-
-                break;
-
-            case INT_ARR:
-                writer.doWriteIntArray((int[])val);
-
-                break;
-
-            case LONG_ARR:
-                writer.doWriteLongArray((long[])val);
-
-                break;
-
-            case FLOAT_ARR:
-                writer.doWriteFloatArray((float[])val);
-
-                break;
-
-            case DOUBLE_ARR:
-                writer.doWriteDoubleArray((double[])val);
-
-                break;
-
-            case CHAR_ARR:
-                writer.doWriteCharArray((char[])val);
-
-                break;
-
-            case BOOLEAN_ARR:
-                writer.doWriteBooleanArray((boolean[])val);
-
-                break;
-
-            case DECIMAL_ARR:
-                writer.doWriteDecimalArray((BigDecimal[])val);
-
-                break;
-
-            case STRING_ARR:
-                writer.doWriteStringArray((String[])val);
-
-                break;
-
-            case UUID_ARR:
-                writer.doWriteUuidArray((UUID[])val);
-
-                break;
-
-            case DATE_ARR:
-                writer.doWriteDateArray((Date[])val);
-
-                break;
-
-            default:
-                throw new IllegalArgumentException("Can't write object with type: " + val.getClass());
-        }
-    }
-
-    /**
-     * @param obj Value to unwrap.
-     * @return Unwrapped value.
-     */
-    public static Object unwrapLazy(@Nullable Object obj) {
-        if (obj instanceof GridPortableLazyValue)
-            return ((GridPortableLazyValue)obj).value();
-
-        return obj;
-    }
-
-    /**
-     * @param delegate Iterator to delegate.
-     * @return New iterator.
-     */
-    public static Iterator<Object> unwrapLazyIterator(final Iterator<Object> delegate) {
-        return new Iterator<Object>() {
-            @Override public boolean hasNext() {
-                return delegate.hasNext();
-            }
-
-            @Override public Object next() {
-                return unwrapLazy(delegate.next());
-            }
-
-            @Override public void remove() {
-                delegate.remove();
-            }
-        };
-    }
-
-    /**
-     * @return {@code true} if content of serialized value cannot contain references to other object.
-     */
-    public static boolean isPlainType(int type) {
-        return type > 0 && type < PLAIN_TYPE_FLAG.length && PLAIN_TYPE_FLAG[type];
-    }
-
-    /**
-     * @param cls Class.
-     * @return Portable field type.
-     */
-    public static byte typeByClass(Class<?> cls) {
-        if (Date.class.isAssignableFrom(cls))
-            return DATE;
-
-        Byte type = PLAIN_CLASS_TO_FLAG.get(cls);
-
-        if (type != null)
-            return type;
-
-        if (cls.isEnum())
-            return ENUM;
-
-        if (cls.isArray())
-            return cls.getComponentType().isEnum() || cls.getComponentType() == Enum.class ? ENUM_ARR : OBJ_ARR;
-
-        if (Collection.class.isAssignableFrom(cls))
-            return COL;
-
-        if (Map.class.isAssignableFrom(cls))
-            return MAP;
-
-        if (Map.Entry.class.isAssignableFrom(cls))
-            return MAP;
-
-        return OBJ;
-    }
-
-    /**
-     * Tells whether provided type is portable or a collection.
-     *
-     * @param cls Class to check.
-     * @return Whether type is portable or a collection.
-     */
-    public static boolean isPortableOrCollectionType(Class<?> cls) {
-        assert cls != null;
-
-        return isPortableType(cls) ||
-            cls == Object[].class ||
-            Collection.class.isAssignableFrom(cls) ||
-            Map.class.isAssignableFrom(cls) ||
-            Map.Entry.class.isAssignableFrom(cls);
-    }
-
-    /**
-     * Tells whether provided type is portable.
-     *
-     * @param cls Class to check.
-     * @return Whether type is portable.
-     */
-    public static boolean isPortableType(Class<?> cls) {
-        assert cls != null;
-
-        return PortableObject.class.isAssignableFrom(cls) ||
-            PORTABLE_CLS.contains(cls) ||
-            cls.isEnum() ||
-            (cls.isArray() && cls.getComponentType().isEnum());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4662feca/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableValueWithType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableValueWithType.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableValueWithType.java
deleted file mode 100644
index 8d696e5..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableValueWithType.java
+++ /dev/null
@@ -1,74 +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.processors.cache.portable.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-/**
- *
- */
-class GridPortableValueWithType implements GridPortableLazyValue {
-    /** */
-    private byte type;
-
-    /** */
-    private Object val;
-
-    /**
-     * @param type Type
-     * @param val Value.
-     */
-    GridPortableValueWithType(byte type, Object val) {
-        this.type = type;
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeTo(GridPortableWriterExImpl writer, GridPortableBuilderSerializer ctx) {
-        if (val instanceof GridPortableBuilderSerializationAware)
-            ((GridPortableBuilderSerializationAware)val).writeTo(writer, ctx);
-        else
-            ctx.writeValue(writer, val);
-    }
-
-    /** {@inheritDoc} */
-    public String typeName() {
-        return CacheObjectPortableProcessorImpl.fieldTypeName(type);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object value() {
-        if (val instanceof GridPortableLazyValue)
-            return ((GridPortableLazyValue)val).value();
-
-        return val;
-    }
-
-    /**
-     * @param val New value.
-     */
-    public void value(Object val) {
-        this.val = val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridPortableValueWithType.class, this);
-    }
-}