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

[21/50] [abbrv] ignite git commit: ignite-1462: hid portable API in 1.4 release

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
new file mode 100644
index 0000000..de0df8d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableMarshaller.java
@@ -0,0 +1,358 @@
+/*
+ * 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.api;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.portable.GridPortableMarshaller;
+import org.apache.ignite.internal.portable.PortableContext;
+import org.apache.ignite.marshaller.AbstractMarshaller;
+import org.apache.ignite.marshaller.MarshallerContext;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableProtocolVersion;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableTypeConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@link org.apache.ignite.marshaller.Marshaller} that lets to serialize and deserialize all objects
+ * in the portable format.
+ * <p>
+ * {@code PortableMarshaller} is tested only on Java HotSpot VM on other VMs it could yield unexpected results.
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * This marshaller has no mandatory configuration parameters.
+ * <h2 class="header">Java Example</h2>
+ * <pre name="code" class="java">
+ * PortableMarshaller marshaller = new PortableMarshaller();
+ *
+ * IgniteConfiguration cfg = new IgniteConfiguration();
+ *
+ * // Override marshaller.
+ * cfg.setMarshaller(marshaller);
+ *
+ * // Starts grid.
+ * G.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * PortableMarshaller can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
+ *     ...
+ *     &lt;property name="marshaller"&gt;
+ *         &lt;bean class="org.apache.ignite.internal.portable.api.PortableMarshaller"&gt;
+ *            ...
+ *         &lt;/bean&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ */
+public class PortableMarshaller extends AbstractMarshaller {
+    /** Default portable protocol version. */
+    public static final PortableProtocolVersion DFLT_PORTABLE_PROTO_VER = PortableProtocolVersion.VER_1_4_0;
+
+    /** Class names. */
+    private Collection<String> clsNames;
+
+    /** ID mapper. */
+    private PortableIdMapper idMapper;
+
+    /** Serializer. */
+    private PortableSerializer serializer;
+
+    /** Types. */
+    private Collection<PortableTypeConfiguration> typeCfgs;
+
+    /** Use timestamp flag. */
+    private boolean useTs = true;
+
+    /** Whether to convert string to bytes using UTF-8 encoding. */
+    private boolean convertString = true;
+
+    /** Meta data enabled flag. */
+    private boolean metaDataEnabled = true;
+
+    /** Keep deserialized flag. */
+    private boolean keepDeserialized = true;
+
+    /** Protocol version. */
+    private PortableProtocolVersion protoVer = DFLT_PORTABLE_PROTO_VER;
+
+    /** */
+    private GridPortableMarshaller impl;
+
+    /**
+     * Gets class names.
+     *
+     * @return Class names.
+     */
+    public Collection<String> getClassNames() {
+        return clsNames;
+    }
+
+    /**
+     * Sets class names of portable objects explicitly.
+     *
+     * @param clsNames Class names.
+     */
+    public void setClassNames(Collection<String> clsNames) {
+        this.clsNames = new ArrayList<>(clsNames.size());
+
+        for (String clsName : clsNames)
+            this.clsNames.add(clsName.trim());
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public PortableIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(PortableIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public PortableSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(PortableSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Gets types configuration.
+     *
+     * @return Types configuration.
+     */
+    public Collection<PortableTypeConfiguration> getTypeConfigurations() {
+        return typeCfgs;
+    }
+
+    /**
+     * Sets type configurations.
+     *
+     * @param typeCfgs Type configurations.
+     */
+    public void setTypeConfigurations(Collection<PortableTypeConfiguration> typeCfgs) {
+        this.typeCfgs = typeCfgs;
+    }
+
+    /**
+     * If {@code true} then date values converted to {@link Timestamp} on deserialization.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public boolean isUseTimestamp() {
+        return useTs;
+    }
+
+    /**
+     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public void setUseTimestamp(boolean useTs) {
+        this.useTs = useTs;
+    }
+
+    /**
+     * Gets strings must be converted to or from bytes using UTF-8 encoding.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @return Flag indicating whether string must be converted to byte array using UTF-8 encoding.
+     */
+    public boolean isConvertStringToBytes() {
+        return convertString;
+    }
+
+    /**
+     * Sets strings must be converted to or from bytes using UTF-8 encoding.
+     * <p>
+     * Default value is {@code true}.
+     *
+     * @param convertString Flag indicating whether string must be converted to byte array using UTF-8 encoding.
+     */
+    public void setConvertStringToBytes(boolean convertString) {
+        this.convertString = convertString;
+    }
+
+    /**
+     * If {@code true}, meta data will be collected or all types. If you need to override this behaviour for
+     * some specific type, use {@link PortableTypeConfiguration#setMetaDataEnabled(Boolean)} method.
+     * <p>
+     * Default value if {@code true}.
+     *
+     * @return Whether meta data is collected.
+     */
+    public boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * If {@code true}, {@link PortableObject} will cache deserialized instance after
+     * {@link PortableObject#deserialize()} is called. All consequent calls of this
+     * method on the same instance of {@link PortableObject} will return that cached
+     * value without actually deserializing portable object. If you need to override this
+     * behaviour for some specific type, use {@link PortableTypeConfiguration#setKeepDeserialized(Boolean)}
+     * method.
+     * <p>
+     * Default value if {@code true}.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /**
+     * Gets portable protocol version.
+     * <p>
+     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
+     *
+     * @return Portable protocol version.
+     */
+    public PortableProtocolVersion getProtocolVersion() {
+        return protoVer;
+    }
+
+    /**
+     * Sets portable protocol version.
+     * <p>
+     * Defaults to {@link #DFLT_PORTABLE_PROTO_VER}.
+     *
+     * @param protoVer Portable protocol version.
+     */
+    public void setProtocolVersion(PortableProtocolVersion protoVer) {
+        if (protoVer == null)
+            throw new IllegalArgumentException("Wrong portable protocol version: " + protoVer);
+
+        this.protoVer = protoVer;
+    }
+
+    /**
+     * Returns currently set {@link MarshallerContext}.
+     *
+     * @return Marshaller context.
+     */
+    public MarshallerContext getContext() {
+        return ctx;
+    }
+
+    /**
+     * Sets {@link PortableContext}.
+     * <p/>
+     * @param ctx Portable context.
+     */
+    private void setPortableContext(PortableContext ctx) {
+        ctx.configure(this);
+
+        impl = new GridPortableMarshaller(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
+        return impl.marshal(obj, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
+        byte[] arr = marshal(obj);
+
+        try {
+            out.write(arr);
+        }
+        catch (IOException e) {
+            throw new PortableException("Failed to marshal the object: " + obj, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(byte[] bytes, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        return impl.deserialize(bytes, clsLdr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        ByteArrayOutputStream buffer = new ByteArrayOutputStream();
+
+        byte[] arr = new byte[4096];
+        int cnt;
+
+        // we have to fully read the InputStream because GridPortableMarshaller requires support of a method that
+        // returns number of bytes remaining.
+        try {
+            while ((cnt = in.read(arr)) != -1)
+                buffer.write(arr, 0, cnt);
+
+            buffer.flush();
+
+            return impl.deserialize(buffer.toByteArray(), clsLdr);
+        }
+        catch (IOException e) {
+            throw new PortableException("Failed to unmarshal the object from InputStream", e);
+        }
+    }
+}

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
new file mode 100644
index 0000000..741c2a5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableProtocolVersion.java
@@ -0,0 +1,41 @@
+/*
+ * 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.api;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Portable protocol version.
+ */
+public enum PortableProtocolVersion {
+    /** Ignite 1.4.0 release. */
+    VER_1_4_0;
+
+    /** Enumerated values. */
+    private static final PortableProtocolVersion[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable public static PortableProtocolVersion fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}
\ No newline at end of file

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

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
new file mode 100644
index 0000000..80a043e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/api/PortableTypeConfiguration.java
@@ -0,0 +1,195 @@
+/*
+ * 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.api;
+
+import java.sql.Timestamp;
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Defines configuration properties for a specific portable type. Providing per-type
+ * configuration is optional, as it is generally enough, and also optional, to provide global portable
+ * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
+ * However, this class allows you to change configuration properties for a specific
+ * portable type without affecting configuration for other portable types.
+ * <p>
+ * Per-type portable configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
+ */
+public class PortableTypeConfiguration {
+    /** Class name. */
+    private String clsName;
+
+    /** ID mapper. */
+    private PortableIdMapper idMapper;
+
+    /** Serializer. */
+    private PortableSerializer serializer;
+
+    /** Use timestamp flag. */
+    private Boolean useTs;
+
+    /** Meta data enabled flag. */
+    private Boolean metaDataEnabled;
+
+    /** Keep deserialized flag. */
+    private Boolean keepDeserialized;
+
+    /** Affinity key field name. */
+    private String affKeyFieldName;
+
+    /**
+     */
+    public PortableTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public PortableTypeConfiguration(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets type name.
+     *
+     * @return Type name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param clsName Type name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public PortableIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(PortableIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public PortableSerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(PortableSerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * If {@code true} then date values converted to {@link Timestamp} during unmarshalling.
+     *
+     * @return Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public Boolean isUseTimestamp() {
+        return useTs;
+    }
+
+    /**
+     * @param useTs Flag indicating whether date values converted to {@link Timestamp} during unmarshalling.
+     */
+    public void setUseTimestamp(Boolean useTs) {
+        this.useTs = useTs;
+    }
+
+    /**
+     * Defines whether meta data is collected for this type. If provided, this value will override
+     * {@link PortableMarshaller#isMetaDataEnabled()} property.
+     *
+     * @return Whether meta data is collected.
+     */
+    public Boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(Boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * Defines whether {@link PortableObject} should cache deserialized instance. If provided,
+     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
+     * property.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /**
+     * Gets affinity key field name.
+     *
+     * @return Affinity key field name.
+     */
+    public String getAffinityKeyFieldName() {
+        return affKeyFieldName;
+    }
+
+    /**
+     * Sets affinity key field name.
+     *
+     * @param affKeyFieldName Affinity key field name.
+     */
+    public void setAffinityKeyFieldName(String affKeyFieldName) {
+        this.affKeyFieldName = affKeyFieldName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(PortableTypeConfiguration.class, this, super.toString());
+    }
+}
\ No newline at end of file

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
index 1472d56..8673b70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderEnum.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.portable.builder;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
index b2e4c0d..96d10a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderImpl.java
@@ -25,17 +25,13 @@ import java.util.Set;
 import org.apache.ignite.internal.processors.cache.portable.CacheObjectPortableProcessorImpl;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import org.apache.ignite.internal.portable.*;
-import org.apache.ignite.internal.processors.cache.portable.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.portable.*;
 
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.CLS_NAME_POS;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.DFLT_HDR_LEN;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
index 45355d7..e93f860 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderReader.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.portable.PortablePrimitives;
 import org.apache.ignite.internal.portable.PortableReaderExImpl;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.api.PortableException;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.ignite.internal.portable.GridPortableMarshaller.NULL;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
index 2d9c961..6fe8875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableBuilderSerializer.java
@@ -21,8 +21,8 @@ import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableObjectEx;
 import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.portable.*;
 
 import java.util.*;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
index d864a6e..15c52e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableEnumArrayLazyValue.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.portable.builder;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
index 1126a3c..96f4944 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortableObjectArrayLazyValue.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.portable.builder;
 import org.apache.ignite.internal.portable.GridPortableMarshaller;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
index 8743fbe..300c4ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/builder/PortablePlainPortableObject.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.portable.builder;
 import org.apache.ignite.internal.portable.PortableObjectImpl;
 import org.apache.ignite.internal.portable.PortableObjectOffheapImpl;
 import org.apache.ignite.internal.portable.PortableWriterExImpl;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
index 107b02e..80f91be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/streams/PortableAbstractInputStream.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.portable.streams;
 
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.api.PortableException;
 
 /**
  * Portable abstract input stream.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 75d4c43..59bb5f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -115,7 +115,7 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.jetbrains.annotations.Nullable;
 
@@ -1063,12 +1063,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration cfg = cacheCtx.config();
 
-        // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
-        if (cfg.isKeepPortableInStore() && cfg.isKeepPortableInStore() != CacheConfiguration.DFLT_KEEP_PORTABLE_IN_STORE
-            && !(ctx.config().getMarshaller() instanceof PortableMarshaller))
-            U.warn(log, "CacheConfiguration.isKeepPortableInStore() configuration property will be ignored because " +
-                "PortableMarshaller is not used");
-
         // Start managers.
         for (GridCacheManager mgr : F.view(cacheCtx.managers(), F.notContains(dhtExcludes(cacheCtx))))
             mgr.start(cacheCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index ce0cdd7..cc6c19a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -311,11 +311,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> IgniteCache<K1, V1> withKeepPortable() {
-        return keepPortable();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withNoRetries() {
         GridCacheGateway<K, V> gate = this.gate;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
index 23edd9e..0dbf71d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheDefaultPortableAffinityKeyMapper.java
@@ -21,7 +21,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
index 2e0d37d..d064601 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableContext.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.portable.PortableUtils;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheDefaultAffinityKeyMapper;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 
 /**
  *

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
index fcd73d2..7f6512b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessor.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.cache.portable;
 import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
index 1be5aea..4cab3db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/CacheObjectPortableProcessorImpl.java
@@ -39,7 +39,7 @@ import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.MutableEntry;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.cache.CacheEntryEventSerializableFilter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -80,11 +80,11 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 import sun.misc.Unsafe;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
index 5ed6505..40c3b70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/portable/IgnitePortablesImpl.java
@@ -18,13 +18,13 @@
 package org.apache.ignite.internal.processors.cache.portable;
 
 import java.util.Collection;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.cacheobject.IgniteCacheObjectProcessor;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableBuilder;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 
 /**