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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 9fb56bc..7d1e14d 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -17,10 +17,24 @@
 
 package org.apache.ignite.configuration;
 
+import java.io.Serializable;
+import java.util.Collection;
+import javax.cache.Cache;
+import javax.cache.configuration.CompleteConfiguration;
+import javax.cache.configuration.Factory;
+import javax.cache.configuration.MutableConfiguration;
+import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheEntryProcessor;
+import org.apache.ignite.cache.CacheInterceptor;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.cache.CacheTypeMetadata;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
 import org.apache.ignite.cache.eviction.EvictionFilter;
@@ -36,15 +50,6 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.plugin.CachePluginConfiguration;
 import org.jetbrains.annotations.Nullable;
 
-import javax.cache.Cache;
-import javax.cache.CacheException;
-import javax.cache.configuration.CompleteConfiguration;
-import javax.cache.configuration.Factory;
-import javax.cache.configuration.MutableConfiguration;
-import javax.cache.expiry.ExpiryPolicy;
-import java.io.Serializable;
-import java.util.Collection;
-
 /**
  * This class defines grid cache configuration. This configuration is passed to
  * grid via {@link IgniteConfiguration#getCacheConfiguration()} method. It defines all configuration
@@ -163,10 +168,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default size for onheap SQL row cache size. */
     public static final int DFLT_SQL_ONHEAP_ROW_CACHE_SIZE = 10 * 1024;
 
-    /** Default value for keep portable in store behavior .*/
-    @SuppressWarnings({"UnnecessaryBoxing", "BooleanConstructorCall"})
-    public static final Boolean DFLT_KEEP_PORTABLE_IN_STORE  = new Boolean(true);
-
     /** Cache name. */
     private String name;
 
@@ -219,9 +220,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private Factory storeFactory;
 
     /** */
-    private Boolean keepPortableInStore = DFLT_KEEP_PORTABLE_IN_STORE;
-
-    /** */
     private boolean loadPrevVal = DFLT_LOAD_PREV_VAL;
 
     /** Node group resolver. */
@@ -383,8 +381,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         invalidate = cc.isInvalidate();
         isReadThrough = cc.isReadThrough();
         isWriteThrough = cc.isWriteThrough();
-        keepPortableInStore = cc.isKeepPortableInStore() != null ? cc.isKeepPortableInStore() :
-            DFLT_KEEP_PORTABLE_IN_STORE;
         listenerConfigurations = cc.listenerConfigurations;
         loadPrevVal = cc.isLoadPreviousValue();
         longQryWarnTimeout = cc.getLongQueryWarningTimeout();
@@ -825,38 +821,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     }
 
     /**
-     * Flag indicating that {@link CacheStore} implementation
-     * is working with portable objects instead of Java objects.
-     * Default value of this flag is {@link #DFLT_KEEP_PORTABLE_IN_STORE},
-     * because this is recommended behavior from performance standpoint.
-     * <p>
-     * If set to {@code false}, Ignite will deserialize keys and
-     * values stored in portable format before they are passed
-     * to cache store.
-     * <p>
-     * Note that setting this flag to {@code false} can simplify
-     * store implementation in some cases, but it can cause performance
-     * degradation due to additional serializations and deserializations
-     * of portable objects. You will also need to have key and value
-     * classes on all nodes since portables will be deserialized when
-     * store is called.
-     *
-     * @return Keep portables in store flag.
-     */
-    public Boolean isKeepPortableInStore() {
-        return keepPortableInStore;
-    }
-
-    /**
-     * Sets keep portables in store flag.
-     *
-     * @param keepPortableInStore Keep portables in store flag.
-     */
-    public void setKeepPortableInStore(boolean keepPortableInStore) {
-        this.keepPortableInStore = keepPortableInStore;
-    }
-
-    /**
      * Gets key topology resolver to provide mapping from keys to nodes.
      *
      * @return Key topology resolver to provide mapping from keys to nodes.
@@ -1860,4 +1824,4 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
             return obj.getClass().equals(this.getClass());
         }
     }
-}
\ 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/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index e3859c5..9443f21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.util.Collection;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.cluster.IgniteClusterEx;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
@@ -140,4 +141,12 @@ public interface IgniteEx extends Ignite {
      * @return Kernal context.
      */
     public GridKernalContext context();
+
+
+    /**
+     * Gets an instance of {@link IgnitePortables} interface.
+     *
+     * @return Instance of {@link IgnitePortables} interface.
+     */
+    public IgnitePortables portables();
 }
\ 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/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 9b615b1..abab1f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -64,7 +64,7 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.IgnitePortables;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -157,7 +157,7 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
@@ -203,7 +203,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_RESTART_ENABLED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_PORT_RANGE;
@@ -1270,9 +1269,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         add(ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
         add(ATTR_USER_NAME, System.getProperty("user.name"));
         add(ATTR_GRID_NAME, gridName);
-        add(ATTR_PORTABLE_PROTO_VER, cfg.getMarshaller() instanceof PortableMarshaller ?
-            ((PortableMarshaller)cfg.getMarshaller()).getProtocolVersion().toString() :
-            PortableMarshaller.DFLT_PORTABLE_PROTO_VER.toString());
 
         add(ATTR_PEER_CLASSLOADING, cfg.isPeerClassLoadingEnabled());
         add(ATTR_DEPLOYMENT_MODE, cfg.getDeploymentMode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 7be2af3..10b8df0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -135,13 +135,10 @@ public final class IgniteNodeAttributes {
     /** Node consistent id. */
     public static final String ATTR_NODE_CONSISTENT_ID = ATTR_PREFIX + ".consistent.id";
 
-    /** Portable protocol version. */
-    public static final String ATTR_PORTABLE_PROTO_VER = ATTR_PREFIX + ".portable.proto.ver";
-
     /**
      * Enforces singleton.
      */
     private IgniteNodeAttributes() {
         /* No-op. */
     }
-}
\ 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/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index bc4f756..3a09b2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -125,7 +125,6 @@ import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PORTABLE_PROTO_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
 import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
@@ -982,8 +981,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         // Fetch local node attributes once.
         String locPreferIpV4 = locNode.attribute("java.net.preferIPv4Stack");
 
-        String locPortableProtoVer = locNode.attribute(ATTR_PORTABLE_PROTO_VER);
-
         Object locMode = locNode.attribute(ATTR_DEPLOYMENT_MODE);
 
         int locJvmMajVer = nodeJavaMajorVersion(locNode);
@@ -1033,13 +1030,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         ", rmtId8=" + U.id8(n.id()) + ", rmtPeerClassLoading=" + rmtP2pEnabled +
                         ", rmtAddrs=" + U.addressesAsString(n) + ']');
             }
-
-            String rmtPortableProtoVer = n.attribute(ATTR_PORTABLE_PROTO_VER);
-
-            // In order to support backward compatibility skip the check for nodes that don't have this attribute.
-            if (rmtPortableProtoVer != null && !F.eq(locPortableProtoVer, rmtPortableProtoVer))
-                throw new IgniteCheckedException("Remote node has portable protocol version different from local " +
-                    "[locVersion=" + locPortableProtoVer + ", rmtVersion=" + rmtPortableProtoVer + ']');
         }
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
index c7a9e6f..4bc8545 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/GridPortableMarshaller.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.portable;
 
 import org.apache.ignite.internal.portable.streams.PortableInputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
-import org.apache.ignite.portable.PortableException;
+import org.apache.ignite.internal.portable.api.PortableException;
 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/portable/PortableClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
index a2b4b74..e1b7324 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableClassDescriptor.java
@@ -40,11 +40,11 @@ import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
 import org.jetbrains.annotations.Nullable;
 
 import static java.lang.reflect.Modifier.isStatic;

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
index 165ad9a..2ee96b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableContext.java
@@ -60,16 +60,16 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
+import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfiguration;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetPortableConfiguration;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetPortableTypeConfiguration;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableIdMapper;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableSerializer;
+import org.apache.ignite.internal.portable.api.PortableTypeConfiguration;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index ae5fbf0..05e7f20 100644
--- 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
@@ -27,9 +27,9 @@ 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.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 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/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..fafafad 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.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
 
 /**
  * Portable meta data handler.

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index 1d26007..c0423eb 100644
--- 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
@@ -27,13 +27,13 @@ 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.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableMarshalAware;
+import org.apache.ignite.internal.portable.api.PortableMetadata;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 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/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
index fe4b628..229c90f 100644
--- 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
@@ -23,9 +23,9 @@ 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.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;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index 47ff1ab..cb81efe 100644
--- 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
@@ -34,9 +34,9 @@ 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.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;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index ba8ee83..e788422 100644
--- 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
@@ -30,9 +30,9 @@ 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.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 sun.misc.Unsafe;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/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
index e703f2f..e401142 100644
--- 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
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.portable;
 
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
 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/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
index a59f157..43b7650 100644
--- 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
@@ -18,8 +18,8 @@
 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.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
 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/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..2537926 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.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/portable/PortableReaderExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
index 4ad125a..a101db5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableReaderExImpl.java
@@ -45,11 +45,11 @@ import org.apache.ignite.internal.util.GridEnumCache;
 import org.apache.ignite.internal.util.lang.GridMapEntry;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableInvalidClassException;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableRawReader;
-import org.apache.ignite.portable.PortableReader;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableInvalidClassException;
+import org.apache.ignite.internal.portable.api.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableRawReader;
+import org.apache.ignite.internal.portable.api.PortableReader;
 import org.jetbrains.annotations.NotNull;
 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/portable/PortableUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
index 7259cc9..ccc1a5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableUtils.java
@@ -35,7 +35,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import org.apache.ignite.internal.portable.builder.PortableLazyValue;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.portable.PortableObject;
+import org.apache.ignite.internal.portable.api.PortableObject;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71379a80/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
index 3152c4b..1d5ca60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/portable/PortableWriterExImpl.java
@@ -18,12 +18,8 @@
 package org.apache.ignite.internal.portable;
 
 import java.io.IOException;
-import java.io.ObjectInputStream;
 import java.io.ObjectOutput;
-import java.io.ObjectOutputStream;
 import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.sql.Timestamp;
@@ -32,14 +28,13 @@ import java.util.Date;
 import java.util.IdentityHashMap;
 import java.util.Map;
 import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.portable.streams.PortableHeapOutputStream;
 import org.apache.ignite.internal.portable.streams.PortableOutputStream;
 import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableRawWriter;
-import org.apache.ignite.portable.PortableWriter;
+import org.apache.ignite.internal.portable.api.PortableException;
+import org.apache.ignite.internal.portable.api.PortableRawWriter;
+import org.apache.ignite.internal.portable.api.PortableWriter;
 import org.jetbrains.annotations.Nullable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;

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

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

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

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

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

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