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:42 UTC

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

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;
     }