You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/12/11 16:56:23 UTC

[55/59] [abbrv] ignite git commit: ignite-2065: rename "portable" classes to "binary"

ignite-2065: rename "portable" classes to "binary"


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

Branch: refs/heads/ignite-843-rc2
Commit: 71ad9ceadee77cf6729db6582980038564ef2dea
Parents: 2b3c93e
Author: ashutak <as...@gridgain.com>
Authored: Fri Dec 11 17:40:51 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Dec 11 17:40:51 2015 +0300

----------------------------------------------------------------------
 .../examples/CacheClientBinaryExampleTest.java  |   46 +
 .../CacheClientPortableExampleTest.java         |   46 -
 .../ComputeClientBinaryExampleTest.java         |   37 +
 .../ComputeClientPortableExampleTest.java       |   37 -
 .../testsuites/IgniteExamplesSelfTestSuite.java |   13 +-
 .../binary/BinaryCachingMetadataHandler.java    |    2 +-
 .../internal/binary/BinaryClassDescriptor.java  |  813 ++++++++
 .../ignite/internal/binary/BinaryContext.java   | 1102 ++++++++++
 .../internal/binary/BinaryEnumObjectImpl.java   |   10 +-
 .../internal/binary/BinaryFieldAccessor.java    |    4 +-
 .../ignite/internal/binary/BinaryFieldImpl.java |   10 +-
 .../internal/binary/BinaryMarshaller.java       |    8 +-
 .../ignite/internal/binary/BinaryMetadata.java  |   12 +-
 .../binary/BinaryMetadataCollector.java         |    8 +-
 .../internal/binary/BinaryObjectExImpl.java     |    2 +-
 .../internal/binary/BinaryObjectImpl.java       |  110 +-
 .../binary/BinaryObjectOffheapImpl.java         |  121 +-
 .../internal/binary/BinaryPositionReadable.java |   47 +
 .../internal/binary/BinaryPrimitives.java       |  382 ++++
 .../internal/binary/BinaryRawWriterEx.java      |    4 +-
 .../internal/binary/BinaryReaderExImpl.java     |  284 +--
 .../ignite/internal/binary/BinarySchema.java    |  466 +++++
 .../internal/binary/BinarySchemaRegistry.java   |  172 ++
 .../binary/BinaryThreadLocalContext.java        |    9 +-
 .../ignite/internal/binary/BinaryTypeImpl.java  |    6 +-
 .../ignite/internal/binary/BinaryUtils.java     | 1859 +++++++++++++++++
 .../ignite/internal/binary/BinaryWriteMode.java |   92 +-
 .../internal/binary/BinaryWriterExImpl.java     |  260 +--
 .../binary/BinaryWriterSchemaHolder.java        |   19 +-
 .../internal/binary/GridBinaryMarshaller.java   |  286 +++
 .../internal/binary/GridPortableMarshaller.java |  289 ---
 .../binary/PortableClassDescriptor.java         |  813 --------
 .../ignite/internal/binary/PortableContext.java | 1102 ----------
 .../binary/PortablePositionReadable.java        |   47 -
 .../internal/binary/PortablePrimitives.java     |  382 ----
 .../ignite/internal/binary/PortableSchema.java  |  466 -----
 .../internal/binary/PortableSchemaRegistry.java |  172 --
 .../ignite/internal/binary/PortableUtils.java   | 1909 ------------------
 .../binary/builder/BinaryAbstractLazyValue.java |   57 +
 .../binary/builder/BinaryBuilderEnum.java       |  115 ++
 .../binary/builder/BinaryBuilderReader.java     |  846 ++++++++
 .../BinaryBuilderSerializationAware.java        |   31 +
 .../binary/builder/BinaryBuilderSerializer.java |  217 ++
 .../builder/BinaryEnumArrayLazyValue.java       |  113 ++
 .../binary/builder/BinaryLazyArrayList.java     |  167 ++
 .../binary/builder/BinaryLazyLinkedList.java    |  218 ++
 .../internal/binary/builder/BinaryLazyMap.java  |  221 ++
 .../internal/binary/builder/BinaryLazySet.java  |   92 +
 .../binary/builder/BinaryLazyValue.java         |   28 +
 .../builder/BinaryModifiableLazyValue.java      |   52 +
 .../builder/BinaryObjectArrayLazyValue.java     |   90 +
 .../binary/builder/BinaryObjectBuilderImpl.java |  120 +-
 .../binary/builder/BinaryPlainBinaryObject.java |   53 +
 .../binary/builder/BinaryPlainLazyValue.java    |   49 +
 .../binary/builder/BinaryValueWithType.java     |   76 +
 .../builder/PortableAbstractLazyValue.java      |   57 -
 .../binary/builder/PortableBuilderEnum.java     |  115 --
 .../binary/builder/PortableBuilderReader.java   |  857 --------
 .../PortableBuilderSerializationAware.java      |   31 -
 .../builder/PortableBuilderSerializer.java      |  217 --
 .../builder/PortableEnumArrayLazyValue.java     |  115 --
 .../binary/builder/PortableLazyArrayList.java   |  167 --
 .../binary/builder/PortableLazyLinkedList.java  |  218 --
 .../binary/builder/PortableLazyMap.java         |  221 --
 .../binary/builder/PortableLazySet.java         |   94 -
 .../binary/builder/PortableLazyValue.java       |   28 -
 .../builder/PortableModifiableLazyValue.java    |   52 -
 .../builder/PortableObjectArrayLazyValue.java   |   90 -
 .../binary/builder/PortablePlainLazyValue.java  |   49 -
 .../builder/PortablePlainPortableObject.java    |   56 -
 .../binary/builder/PortableValueWithType.java   |   77 -
 .../streams/BinaryAbstractInputStream.java      |  379 ++++
 .../streams/BinaryAbstractOutputStream.java     |  347 ++++
 .../binary/streams/BinaryAbstractStream.java    |   80 +
 .../binary/streams/BinaryHeapInputStream.java   |  166 ++
 .../binary/streams/BinaryHeapOutputStream.java  |  176 ++
 .../binary/streams/BinaryInputStream.java       |  162 ++
 .../binary/streams/BinaryMemoryAllocator.java   |   57 +
 .../streams/BinaryMemoryAllocatorChunk.java     |  117 ++
 .../streams/BinaryOffheapInputStream.java       |  144 ++
 .../streams/BinaryOffheapOutputStream.java      |  222 ++
 .../binary/streams/BinaryOutputStream.java      |  259 +++
 .../internal/binary/streams/BinaryStream.java   |   53 +
 .../streams/PortableAbstractInputStream.java    |  379 ----
 .../streams/PortableAbstractOutputStream.java   |  347 ----
 .../binary/streams/PortableAbstractStream.java  |   80 -
 .../binary/streams/PortableHeapInputStream.java |  166 --
 .../streams/PortableHeapOutputStream.java       |  176 --
 .../binary/streams/PortableInputStream.java     |  163 --
 .../binary/streams/PortableMemoryAllocator.java |   57 -
 .../streams/PortableMemoryAllocatorChunk.java   |  117 --
 .../streams/PortableOffheapInputStream.java     |  144 --
 .../streams/PortableOffheapOutputStream.java    |  222 --
 .../binary/streams/PortableOutputStream.java    |  259 ---
 .../internal/binary/streams/PortableStream.java |   53 -
 .../processors/cache/CacheObjectContext.java    |    6 +-
 .../cache/binary/BinaryMetadataKey.java         |   82 +
 .../CacheDefaultBinaryAffinityKeyMapper.java    |   51 +
 .../CacheDefaultPortableAffinityKeyMapper.java  |   51 -
 .../cache/binary/CacheObjectBinaryContext.java  |   56 +
 .../binary/CacheObjectBinaryProcessorImpl.java  |  104 +-
 .../binary/CacheObjectPortableContext.java      |   56 -
 .../cache/binary/PortableMetadataKey.java       |   82 -
 .../platform/PlatformContextImpl.java           |    4 +-
 .../PlatformDotNetConfigurationClosure.java     |   10 +-
 .../platform/memory/PlatformInputStream.java    |    4 +-
 .../platform/memory/PlatformOutputStream.java   |    4 +-
 .../message/GridClientBinaryMetaData.java       |   71 +
 .../message/GridClientPortableMetaData.java     |   71 -
 .../binary/BinaryFieldsAbstractSelfTest.java    |    6 +-
 .../BinaryFooterOffsetsAbstractSelfTest.java    |    4 +-
 .../binary/BinaryMarshallerSelfTest.java        |   24 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |   10 +-
 .../binary/GridBinaryAffinityKeySelfTest.java   |  234 +++
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |  247 +++
 .../binary/GridPortableAffinityKeySelfTest.java |  234 ---
 ...idPortableMarshallerCtxDisabledSelfTest.java |  247 ---
 .../binary/GridPortableWildcardsSelfTest.java   |   30 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   48 +
 ...ryDuplicateIndexObjectsAbstractSelfTest.java |  161 ++
 .../GridCacheBinaryStoreAbstractSelfTest.java   |  300 +++
 .../GridCacheBinaryStoreObjectsSelfTest.java    |   55 +
 .../GridCacheBinaryStorePortablesSelfTest.java  |   66 +
 .../GridCachePortableStoreAbstractSelfTest.java |  300 ---
 .../GridCachePortableStoreObjectsSelfTest.java  |   55 -
 ...GridCachePortableStorePortablesSelfTest.java |   66 -
 ...ridPortableCacheEntryMemorySizeSelfTest.java |   48 -
 ...leDuplicateIndexObjectsAbstractSelfTest.java |  161 --
 .../DataStreamProcessorBinarySelfTest.java      |   71 +
 .../DataStreamProcessorPortableSelfTest.java    |   71 -
 .../GridCacheAffinityRoutingBinarySelfTest.java |   54 +
 ...ridCacheAffinityRoutingPortableSelfTest.java |   54 -
 ...OnlyBinaryDataStreamerMultiNodeSelfTest.java |   29 +
 ...BinaryDataStreamerMultithreadedSelfTest.java |   47 +
 ...cPartitionedOnlyBinaryMultiNodeSelfTest.java |   28 +
 ...titionedOnlyBinaryMultithreadedSelfTest.java |   47 +
 ...lyPortableDataStreamerMultiNodeSelfTest.java |   29 -
 ...rtableDataStreamerMultithreadedSelfTest.java |   47 -
 ...artitionedOnlyPortableMultiNodeSelfTest.java |   28 -
 ...tionedOnlyPortableMultithreadedSelfTest.java |   47 -
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 +
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 +
 ...xObjectPartitionedTransactionalSelfTest.java |   41 +
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |   36 +
 .../GridCacheMemoryModePortableSelfTest.java    |   36 -
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |   48 +
 ...acheOffHeapTieredAtomicPortableSelfTest.java |   48 -
 .../GridCacheOffHeapTieredBinarySelfTest.java   |   48 +
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |   96 +
 ...eapTieredEvictionAtomicPortableSelfTest.java |   96 -
 ...acheOffHeapTieredEvictionBinarySelfTest.java |   96 +
 ...heOffHeapTieredEvictionPortableSelfTest.java |   96 -
 .../GridCacheOffHeapTieredPortableSelfTest.java |   48 -
 ...ateIndexObjectPartitionedAtomicSelfTest.java |   38 -
 ...xObjectPartitionedTransactionalSelfTest.java |   41 -
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |   42 -
 .../session/GridSessionCheckpointSelfTest.java  |    4 +-
 .../ignite/testframework/junits/IgniteMock.java |    6 +-
 .../junits/IgniteTestResources.java             |    4 +-
 .../IgnitePortableCacheTestSuite.java           |   52 +-
 .../IgnitePortableObjectsTestSuite.java         |   16 +-
 .../CacheDeploymentBinaryEntryProcessor.java    |   35 +
 .../CacheDeploymentPortableEntryProcessor.java  |   35 -
 .../IgnitePortableCacheQueryTestSuite.java      |    8 +-
 164 files changed, 12582 insertions(+), 12729 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/examples/src/test/java/org/apache/ignite/examples/CacheClientBinaryExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheClientBinaryExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheClientBinaryExampleTest.java
new file mode 100644
index 0000000..d5f8cc0
--- /dev/null
+++ b/examples/src/test/java/org/apache/ignite/examples/CacheClientBinaryExampleTest.java
@@ -0,0 +1,46 @@
+/*
+ * 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.examples;
+
+import org.apache.ignite.examples.binary.datagrid.CacheClientBinaryPutGetExample;
+import org.apache.ignite.examples.binary.datagrid.CacheClientBinaryQueryExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
+/**
+ *
+ */
+public class CacheClientBinaryExampleTest extends GridAbstractExamplesTest {
+    /** {@inheritDoc} */
+    @Override protected String defaultConfig() {
+        return "examples/config/portable/example-ignite-portable.xml";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortablePutGetExample() throws Exception {
+        CacheClientBinaryPutGetExample.main(new String[] {});
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableQueryExample() throws Exception {
+        CacheClientBinaryQueryExample.main(new String[] {});
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
deleted file mode 100644
index 22261e2..0000000
--- a/examples/src/test/java/org/apache/ignite/examples/CacheClientPortableExampleTest.java
+++ /dev/null
@@ -1,46 +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.examples;
-
-import org.apache.ignite.examples.binary.datagrid.CacheClientBinaryPutGetExample;
-import org.apache.ignite.examples.binary.datagrid.CacheClientBinaryQueryExample;
-import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
-
-/**
- *
- */
-public class CacheClientPortableExampleTest extends GridAbstractExamplesTest {
-    /** {@inheritDoc} */
-    @Override protected String defaultConfig() {
-        return "examples/config/portable/example-ignite-portable.xml";
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortablePutGetExample() throws Exception {
-        CacheClientBinaryPutGetExample.main(new String[] {});
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableQueryExample() throws Exception {
-        CacheClientBinaryQueryExample.main(new String[] {});
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/examples/src/test/java/org/apache/ignite/examples/ComputeClientBinaryExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/ComputeClientBinaryExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/ComputeClientBinaryExampleTest.java
new file mode 100644
index 0000000..bdba7c3
--- /dev/null
+++ b/examples/src/test/java/org/apache/ignite/examples/ComputeClientBinaryExampleTest.java
@@ -0,0 +1,37 @@
+/*
+ * 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.examples;
+
+import org.apache.ignite.examples.binary.computegrid.ComputeClientBinaryTaskExecutionExample;
+import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
+
+/**
+ *
+ */
+public class ComputeClientBinaryExampleTest extends GridAbstractExamplesTest {
+    /** {@inheritDoc} */
+    @Override protected String defaultConfig() {
+        return "examples/config/portable/example-ignite-portable.xml";
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPortableTaskExecutionExample() throws Exception {
+        ComputeClientBinaryTaskExecutionExample.main(new String[] {});
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java b/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
deleted file mode 100644
index 44d8776..0000000
--- a/examples/src/test/java/org/apache/ignite/examples/ComputeClientPortableExampleTest.java
+++ /dev/null
@@ -1,37 +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.examples;
-
-import org.apache.ignite.examples.binary.computegrid.ComputeClientBinaryTaskExecutionExample;
-import org.apache.ignite.testframework.junits.common.GridAbstractExamplesTest;
-
-/**
- *
- */
-public class ComputeClientPortableExampleTest extends GridAbstractExamplesTest {
-    /** {@inheritDoc} */
-    @Override protected String defaultConfig() {
-        return "examples/config/portable/example-ignite-portable.xml";
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPortableTaskExecutionExample() throws Exception {
-        ComputeClientBinaryTaskExecutionExample.main(new String[] {});
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java b/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
index 4412af3..54fa8a3 100644
--- a/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
+++ b/examples/src/test/java/org/apache/ignite/testsuites/IgniteExamplesSelfTestSuite.java
@@ -20,12 +20,12 @@ package org.apache.ignite.testsuites;
 import junit.framework.TestSuite;
 import org.apache.ignite.examples.BasicExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.BasicExamplesSelfTest;
-import org.apache.ignite.examples.CacheClientPortableExampleTest;
+import org.apache.ignite.examples.CacheClientBinaryExampleTest;
 import org.apache.ignite.examples.CacheExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.CacheExamplesSelfTest;
 import org.apache.ignite.examples.CheckpointExamplesSelfTest;
 import org.apache.ignite.examples.ClusterGroupExampleSelfTest;
-import org.apache.ignite.examples.ComputeClientPortableExampleTest;
+import org.apache.ignite.examples.ComputeClientBinaryExampleTest;
 import org.apache.ignite.examples.ContinuationExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.ContinuationExamplesSelfTest;
 import org.apache.ignite.examples.ContinuousMapperExamplesMultiNodeSelfTest;
@@ -44,9 +44,6 @@ import org.apache.ignite.examples.MonteCarloExamplesSelfTest;
 import org.apache.ignite.examples.SpringBeanExamplesSelfTest;
 import org.apache.ignite.examples.TaskExamplesMultiNodeSelfTest;
 import org.apache.ignite.examples.TaskExamplesSelfTest;
-import org.apache.ignite.testframework.GridTestUtils;
-
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_OVERRIDE_MCAST_GRP;
 
 /**
  * Examples test suite.
@@ -92,9 +89,9 @@ public class IgniteExamplesSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(MonteCarloExamplesMultiNodeSelfTest.class));
 
         // Portable.
-        suite.addTest(new TestSuite(CacheClientPortableExampleTest.class));
-        suite.addTest(new TestSuite(ComputeClientPortableExampleTest.class));
+        suite.addTest(new TestSuite(CacheClientBinaryExampleTest.class));
+        suite.addTest(new TestSuite(ComputeClientBinaryExampleTest.class));
 
         return suite;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
index 584b683..39189f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryCachingMetadataHandler.java
@@ -54,7 +54,7 @@ public class BinaryCachingMetadataHandler implements BinaryMetadataHandler {
                 BinaryMetadata oldMeta = ((BinaryTypeImpl)oldType).metadata();
                 BinaryMetadata newMeta = ((BinaryTypeImpl)type).metadata();
 
-                BinaryMetadata mergedMeta = PortableUtils.mergeMetadata(oldMeta, newMeta);
+                BinaryMetadata mergedMeta = BinaryUtils.mergeMetadata(oldMeta, newMeta);
 
                 BinaryType mergedType = mergedMeta.wrap(((BinaryTypeImpl)oldType).context());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
new file mode 100644
index 0000000..0c3275e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
@@ -0,0 +1,813 @@
+/*
+ * 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.binary;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.Binarylizable;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerExclusions;
+import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
+import org.jetbrains.annotations.Nullable;
+import sun.misc.Unsafe;
+
+import static java.lang.reflect.Modifier.isStatic;
+import static java.lang.reflect.Modifier.isTransient;
+
+/**
+ * Portable class descriptor.
+ */
+public class BinaryClassDescriptor {
+    /** */
+    public static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    private final BinaryContext ctx;
+
+    /** */
+    private final Class<?> cls;
+
+    /** */
+    private final BinarySerializer serializer;
+
+    /** ID mapper. */
+    private final BinaryIdMapper idMapper;
+
+    /** */
+    private final BinaryWriteMode mode;
+
+    /** */
+    private final boolean userType;
+
+    /** */
+    private final int typeId;
+
+    /** */
+    private final String typeName;
+
+    /** Affinity key field name. */
+    private final String affKeyFieldName;
+
+    /** */
+    private final Constructor<?> ctor;
+
+    /** */
+    private final BinaryFieldAccessor[] fields;
+
+    /** */
+    private final Method writeReplaceMtd;
+
+    /** */
+    private final Method readResolveMtd;
+
+    /** */
+    private final Map<String, Integer> stableFieldsMeta;
+
+    /** Object schemas. Initialized only for serializable classes and contains only 1 entry. */
+    private final BinarySchema stableSchema;
+
+    /** Schema registry. */
+    private final BinarySchemaRegistry schemaReg;
+
+    /** */
+    private final boolean registered;
+
+    /** */
+    private final boolean useOptMarshaller;
+
+    /** */
+    private final boolean excluded;
+
+    /**
+     * @param ctx Context.
+     * @param cls Class.
+     * @param userType User type flag.
+     * @param typeId Type ID.
+     * @param typeName Type name.
+     * @param affKeyFieldName Affinity key field name.
+     * @param idMapper ID mapper.
+     * @param serializer Serializer.
+     * @param metaDataEnabled Metadata enabled flag.
+     * @param registered Whether typeId has been successfully registered by MarshallerContext or not.
+     * @param predefined Whether the class is predefined or not.
+     * @throws BinaryObjectException In case of error.
+     */
+    BinaryClassDescriptor(
+        BinaryContext ctx,
+        Class<?> cls,
+        boolean userType,
+        int typeId,
+        String typeName,
+        @Nullable String affKeyFieldName,
+        @Nullable BinaryIdMapper idMapper,
+        @Nullable BinarySerializer serializer,
+        boolean metaDataEnabled,
+        boolean registered,
+        boolean predefined
+    ) throws BinaryObjectException {
+        assert ctx != null;
+        assert cls != null;
+        assert idMapper != null;
+
+        this.ctx = ctx;
+        this.cls = cls;
+        this.typeId = typeId;
+        this.userType = userType;
+        this.typeName = typeName;
+        this.affKeyFieldName = affKeyFieldName;
+        this.serializer = serializer;
+        this.idMapper = idMapper;
+        this.registered = registered;
+
+        schemaReg = ctx.schemaRegistry(typeId);
+
+        excluded = MarshallerExclusions.isExcluded(cls);
+
+        useOptMarshaller = !predefined && initUseOptimizedMarshallerFlag();
+
+        if (excluded)
+            mode = BinaryWriteMode.EXCLUSION;
+        else {
+            if (cls == BinaryEnumObjectImpl.class)
+                mode = BinaryWriteMode.PORTABLE_ENUM;
+            else
+                mode = serializer != null ? BinaryWriteMode.PORTABLE : BinaryUtils.mode(cls);
+        }
+
+        switch (mode) {
+            case P_BYTE:
+            case P_BOOLEAN:
+            case P_SHORT:
+            case P_CHAR:
+            case P_INT:
+            case P_LONG:
+            case P_FLOAT:
+            case P_DOUBLE:
+            case BYTE:
+            case SHORT:
+            case INT:
+            case LONG:
+            case FLOAT:
+            case DOUBLE:
+            case CHAR:
+            case BOOLEAN:
+            case DECIMAL:
+            case STRING:
+            case UUID:
+            case DATE:
+            case TIMESTAMP:
+            case BYTE_ARR:
+            case SHORT_ARR:
+            case INT_ARR:
+            case LONG_ARR:
+            case FLOAT_ARR:
+            case DOUBLE_ARR:
+            case CHAR_ARR:
+            case BOOLEAN_ARR:
+            case DECIMAL_ARR:
+            case STRING_ARR:
+            case UUID_ARR:
+            case DATE_ARR:
+            case TIMESTAMP_ARR:
+            case OBJECT_ARR:
+            case COL:
+            case MAP:
+            case PORTABLE_OBJ:
+            case ENUM:
+            case PORTABLE_ENUM:
+            case ENUM_ARR:
+            case CLASS:
+            case EXCLUSION:
+                ctor = null;
+                fields = null;
+                stableFieldsMeta = null;
+                stableSchema = null;
+
+                break;
+
+            case PORTABLE:
+            case EXTERNALIZABLE:
+                ctor = constructor(cls);
+                fields = null;
+                stableFieldsMeta = null;
+                stableSchema = null;
+
+                break;
+
+            case OBJECT:
+                // Must not use constructor to honor transient fields semantics.
+                ctor = null;
+                ArrayList<BinaryFieldAccessor> fields0 = new ArrayList<>();
+                stableFieldsMeta = metaDataEnabled ? new HashMap<String, Integer>() : null;
+
+                BinarySchema.Builder schemaBuilder = BinarySchema.Builder.newBuilder();
+
+                Collection<String> names = new HashSet<>();
+                Collection<Integer> ids = new HashSet<>();
+
+                for (Class<?> c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+                    for (Field f : c.getDeclaredFields()) {
+                        int mod = f.getModifiers();
+
+                        if (!isStatic(mod) && !isTransient(mod)) {
+                            f.setAccessible(true);
+
+                            String name = f.getName();
+
+                            if (!names.add(name))
+                                throw new BinaryObjectException("Duplicate field name [fieldName=" + name +
+                                    ", cls=" + cls.getName() + ']');
+
+                            int fieldId = idMapper.fieldId(typeId, name);
+
+                            if (!ids.add(fieldId))
+                                throw new BinaryObjectException("Duplicate field ID: " + name);
+
+                            BinaryFieldAccessor fieldInfo = BinaryFieldAccessor.create(f, fieldId);
+
+                            fields0.add(fieldInfo);
+
+                            schemaBuilder.addField(fieldId);
+
+                            if (metaDataEnabled)
+                                stableFieldsMeta.put(name, fieldInfo.mode().typeId());
+                        }
+                    }
+                }
+
+                fields = fields0.toArray(new BinaryFieldAccessor[fields0.size()]);
+
+                stableSchema = schemaBuilder.build();
+
+                break;
+
+            default:
+                // Should never happen.
+                throw new BinaryObjectException("Invalid mode: " + mode);
+        }
+
+        if (mode == BinaryWriteMode.PORTABLE || mode == BinaryWriteMode.EXTERNALIZABLE ||
+            mode == BinaryWriteMode.OBJECT) {
+            readResolveMtd = U.findNonPublicMethod(cls, "readResolve");
+            writeReplaceMtd = U.findNonPublicMethod(cls, "writeReplace");
+        }
+        else {
+            readResolveMtd = null;
+            writeReplaceMtd = null;
+        }
+    }
+
+    /**
+     * @return {@code True} if enum.
+     */
+    boolean isEnum() {
+        return mode == BinaryWriteMode.ENUM;
+    }
+
+    /**
+     * @return Described class.
+     */
+    Class<?> describedClass() {
+        return cls;
+    }
+
+    /**
+     * @return Type ID.
+     */
+    public int typeId() {
+        return typeId;
+    }
+
+    /**
+     * @return User type flag.
+     */
+    public boolean userType() {
+        return userType;
+    }
+
+    /**
+     * @return Fields meta data.
+     */
+    Map<String, Integer> fieldsMeta() {
+        return stableFieldsMeta;
+    }
+
+    /**
+     * @return Schema.
+     */
+    BinarySchema schema() {
+        return stableSchema;
+    }
+
+    /**
+     * @return Whether typeId has been successfully registered by MarshallerContext or not.
+     */
+    public boolean registered() {
+        return registered;
+    }
+
+    /**
+     * @return {@code true} if {@link OptimizedMarshaller} must be used instead of {@link BinaryMarshaller}
+     * for object serialization and deserialization.
+     */
+    public boolean useOptimizedMarshaller() {
+        return useOptMarshaller;
+    }
+
+    /**
+     * Checks whether the class values are explicitly excluded from marshalling.
+     *
+     * @return {@code true} if excluded, {@code false} otherwise.
+     */
+    public boolean excluded() {
+        return excluded;
+    }
+
+    /**
+     * @return portableWriteReplace() method
+     */
+    @Nullable Method getWriteReplaceMethod() {
+        return writeReplaceMtd;
+    }
+
+    /**
+     * @return portableReadResolve() method
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    @Nullable Method getReadResolveMethod() {
+        return readResolveMtd;
+    }
+
+    /**
+     * @param obj Object.
+     * @param writer Writer.
+     * @throws BinaryObjectException In case of error.
+     */
+    void write(Object obj, BinaryWriterExImpl writer) throws BinaryObjectException {
+        assert obj != null;
+        assert writer != null;
+
+        writer.typeId(typeId);
+
+        switch (mode) {
+            case P_BYTE:
+            case BYTE:
+                writer.writeByteFieldPrimitive((byte) obj);
+
+                break;
+
+            case P_SHORT:
+            case SHORT:
+                writer.writeShortFieldPrimitive((short)obj);
+
+                break;
+
+            case P_INT:
+            case INT:
+                writer.writeIntFieldPrimitive((int) obj);
+
+                break;
+
+            case P_LONG:
+            case LONG:
+                writer.writeLongFieldPrimitive((long) obj);
+
+                break;
+
+            case P_FLOAT:
+            case FLOAT:
+                writer.writeFloatFieldPrimitive((float) obj);
+
+                break;
+
+            case P_DOUBLE:
+            case DOUBLE:
+                writer.writeDoubleFieldPrimitive((double) obj);
+
+                break;
+
+            case P_CHAR:
+            case CHAR:
+                writer.writeCharFieldPrimitive((char) obj);
+
+                break;
+
+            case P_BOOLEAN:
+            case BOOLEAN:
+                writer.writeBooleanFieldPrimitive((boolean) obj);
+
+                break;
+
+            case DECIMAL:
+                writer.doWriteDecimal((BigDecimal)obj);
+
+                break;
+
+            case STRING:
+                writer.doWriteString((String)obj);
+
+                break;
+
+            case UUID:
+                writer.doWriteUuid((UUID)obj);
+
+                break;
+
+            case DATE:
+                writer.doWriteDate((Date)obj);
+
+                break;
+
+            case TIMESTAMP:
+                writer.doWriteTimestamp((Timestamp)obj);
+
+                break;
+
+            case BYTE_ARR:
+                writer.doWriteByteArray((byte[])obj);
+
+                break;
+
+            case SHORT_ARR:
+                writer.doWriteShortArray((short[]) obj);
+
+                break;
+
+            case INT_ARR:
+                writer.doWriteIntArray((int[]) obj);
+
+                break;
+
+            case LONG_ARR:
+                writer.doWriteLongArray((long[]) obj);
+
+                break;
+
+            case FLOAT_ARR:
+                writer.doWriteFloatArray((float[]) obj);
+
+                break;
+
+            case DOUBLE_ARR:
+                writer.doWriteDoubleArray((double[]) obj);
+
+                break;
+
+            case CHAR_ARR:
+                writer.doWriteCharArray((char[]) obj);
+
+                break;
+
+            case BOOLEAN_ARR:
+                writer.doWriteBooleanArray((boolean[]) obj);
+
+                break;
+
+            case DECIMAL_ARR:
+                writer.doWriteDecimalArray((BigDecimal[]) obj);
+
+                break;
+
+            case STRING_ARR:
+                writer.doWriteStringArray((String[]) obj);
+
+                break;
+
+            case UUID_ARR:
+                writer.doWriteUuidArray((UUID[]) obj);
+
+                break;
+
+            case DATE_ARR:
+                writer.doWriteDateArray((Date[]) obj);
+
+                break;
+
+            case TIMESTAMP_ARR:
+                writer.doWriteTimestampArray((Timestamp[]) obj);
+
+                break;
+
+            case OBJECT_ARR:
+                writer.doWriteObjectArray((Object[])obj);
+
+                break;
+
+            case COL:
+                writer.doWriteCollection((Collection<?>)obj);
+
+                break;
+
+            case MAP:
+                writer.doWriteMap((Map<?, ?>)obj);
+
+                break;
+
+            case ENUM:
+                writer.doWriteEnum((Enum<?>)obj);
+
+                break;
+
+            case PORTABLE_ENUM:
+                writer.doWritePortableEnum((BinaryEnumObjectImpl)obj);
+
+                break;
+
+            case ENUM_ARR:
+                writer.doWriteEnumArray((Object[])obj);
+
+                break;
+
+            case CLASS:
+                writer.doWriteClass((Class)obj);
+
+                break;
+
+            case PORTABLE_OBJ:
+                writer.doWritePortableObject((BinaryObjectImpl)obj);
+
+                break;
+
+            case PORTABLE:
+                if (preWrite(writer, obj)) {
+                    try {
+                        if (serializer != null)
+                            serializer.writeBinary(obj, writer);
+                        else
+                            ((Binarylizable)obj).writeBinary(writer);
+
+                        postWrite(writer, obj);
+
+                        // Check whether we need to update metadata.
+                        if (obj.getClass() != BinaryMetadata.class) {
+                            int schemaId = writer.schemaId();
+
+                            if (schemaReg.schema(schemaId) == null) {
+                                // This is new schema, let's update metadata.
+                                BinaryMetadataCollector collector =
+                                    new BinaryMetadataCollector(typeId, typeName, idMapper);
+
+                                if (serializer != null)
+                                    serializer.writeBinary(obj, collector);
+                                else
+                                    ((Binarylizable)obj).writeBinary(collector);
+
+                                BinarySchema newSchema = collector.schema();
+
+                                BinaryMetadata meta = new BinaryMetadata(typeId, typeName, collector.meta(),
+                                    affKeyFieldName, Collections.singleton(newSchema), false);
+
+                                ctx.updateMetadata(typeId, meta);
+
+                                schemaReg.addSchema(newSchema.schemaId(), newSchema);
+                            }
+                        }
+                    }
+                    finally {
+                        writer.popSchema();
+                    }
+                }
+
+                break;
+
+            case EXTERNALIZABLE:
+                if (preWrite(writer, obj)) {
+                    writer.rawWriter();
+
+                    try {
+                        ((Externalizable)obj).writeExternal(writer);
+
+                        postWrite(writer, obj);
+                    }
+                    catch (IOException e) {
+                        throw new BinaryObjectException("Failed to write Externalizable object: " + obj, e);
+                    }
+                    finally {
+                        writer.popSchema();
+                    }
+                }
+
+                break;
+
+            case OBJECT:
+                if (preWrite(writer, obj)) {
+                    try {
+                        for (BinaryFieldAccessor info : fields)
+                            info.write(obj, writer);
+
+                        writer.schemaId(stableSchema.schemaId());
+
+                        postWrite(writer, obj);
+                    }
+                    finally {
+                        writer.popSchema();
+                    }
+                }
+
+                break;
+
+            default:
+                assert false : "Invalid mode: " + mode;
+        }
+    }
+
+    /**
+     * @param reader Reader.
+     * @return Object.
+     * @throws BinaryObjectException If failed.
+     */
+    Object read(BinaryReaderExImpl reader) throws BinaryObjectException {
+        assert reader != null;
+
+        Object res;
+
+        switch (mode) {
+            case PORTABLE:
+                res = newInstance();
+
+                reader.setHandle(res);
+
+                if (serializer != null)
+                    serializer.readBinary(res, reader);
+                else
+                    ((Binarylizable)res).readBinary(reader);
+
+                break;
+
+            case EXTERNALIZABLE:
+                res = newInstance();
+
+                reader.setHandle(res);
+
+                try {
+                    ((Externalizable)res).readExternal(reader);
+                }
+                catch (IOException | ClassNotFoundException e) {
+                    throw new BinaryObjectException("Failed to read Externalizable object: " +
+                        res.getClass().getName(), e);
+                }
+
+                break;
+
+            case OBJECT:
+                res = newInstance();
+
+                reader.setHandle(res);
+
+                for (BinaryFieldAccessor info : fields)
+                    info.read(res, reader);
+
+                break;
+
+            default:
+                assert false : "Invalid mode: " + mode;
+
+                return null;
+        }
+
+        if (readResolveMtd != null) {
+            try {
+                res = readResolveMtd.invoke(res);
+
+                reader.setHandle(res);
+            }
+            catch (IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+            catch (InvocationTargetException e) {
+                if (e.getTargetException() instanceof BinaryObjectException)
+                    throw (BinaryObjectException)e.getTargetException();
+
+                throw new BinaryObjectException("Failed to execute readResolve() method on " + res, e);
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Pre-write phase.
+     *
+     * @param writer Writer.
+     * @param obj Object.
+     * @return Whether further write is needed.
+     */
+    private boolean preWrite(BinaryWriterExImpl writer, Object obj) {
+        if (writer.tryWriteAsHandle(obj))
+            return false;
+
+        writer.preWrite(registered ? null : cls.getName());
+
+        return true;
+    }
+
+    /**
+     * Post-write phase.
+     *
+     * @param writer Writer.
+     * @param obj Object.
+     */
+    private void postWrite(BinaryWriterExImpl writer, Object obj) {
+        writer.postWrite(userType, registered, obj instanceof CacheObjectImpl ? 0 : obj.hashCode());
+    }
+
+    /**
+     * @return Instance.
+     * @throws BinaryObjectException In case of error.
+     */
+    private Object newInstance() throws BinaryObjectException {
+        try {
+            return ctor != null ? ctor.newInstance() : UNSAFE.allocateInstance(cls);
+        }
+        catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
+            throw new BinaryObjectException("Failed to instantiate instance: " + cls, e);
+        }
+    }
+
+    /**
+     * @param cls Class.
+     * @return Constructor.
+     * @throws BinaryObjectException If constructor doesn't exist.
+     */
+    @SuppressWarnings("ConstantConditions")
+    @Nullable private static Constructor<?> constructor(Class<?> cls) throws BinaryObjectException {
+        assert cls != null;
+
+        try {
+            Constructor<?> ctor = U.forceEmptyConstructor(cls);
+
+            if (ctor == null)
+                throw new BinaryObjectException("Failed to find empty constructor for class: " + cls.getName());
+
+            ctor.setAccessible(true);
+
+            return ctor;
+        }
+        catch (IgniteCheckedException e) {
+            throw new BinaryObjectException("Failed to get constructor for class: " + cls.getName(), e);
+        }
+    }
+
+    /**
+     * Determines whether to use {@link OptimizedMarshaller} for serialization or
+     * not.
+     *
+     * @return {@code true} if to use, {@code false} otherwise.
+     */
+    @SuppressWarnings("unchecked")
+    private boolean initUseOptimizedMarshallerFlag() {
+        for (Class c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+            try {
+                Method writeObj = c.getDeclaredMethod("writeObject", ObjectOutputStream.class);
+                Method readObj = c.getDeclaredMethod("readObject", ObjectInputStream.class);
+
+                if (!Modifier.isStatic(writeObj.getModifiers()) && !Modifier.isStatic(readObj.getModifiers()) &&
+                    writeObj.getReturnType() == void.class && readObj.getReturnType() == void.class)
+                    return true;
+            }
+            catch (NoSuchMethodException ignored) {
+                // No-op.
+            }
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
new file mode 100644
index 0000000..6293cfe
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -0,0 +1,1102 @@
+/*
+ * 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.binary;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryIdMapper;
+import org.apache.ignite.binary.BinaryInvalidTypeException;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinarySerializer;
+import org.apache.ignite.binary.BinaryType;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.cache.CacheKeyConfiguration;
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.IgnitionEx;
+import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+import org.apache.ignite.internal.processors.datastructures.CollocatedQueueItemKey;
+import org.apache.ignite.internal.processors.datastructures.CollocatedSetItemKey;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.lang.GridMapEntry;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.T2;
+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.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.ObjectStreamException;
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedHashSet;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import java.util.jar.JarEntry;
+import java.util.jar.JarFile;
+
+/**
+ * Portable context.
+ */
+public class BinaryContext implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final ClassLoader dfltLdr = U.gridClassLoader();
+
+    /** */
+    private final ConcurrentMap<Class<?>, BinaryClassDescriptor> descByCls = new ConcurrentHashMap8<>();
+
+    /** Holds classes loaded by default class loader only. */
+    private final ConcurrentMap<Integer, BinaryClassDescriptor> userTypes = new ConcurrentHashMap8<>();
+
+    /** */
+    private final Map<Integer, BinaryClassDescriptor> predefinedTypes = new HashMap<>();
+
+    /** */
+    private final Map<String, Integer> predefinedTypeNames = new HashMap<>();
+
+    /** */
+    private final Map<Class<? extends Collection>, Byte> colTypes = new HashMap<>();
+
+    /** */
+    private final Map<Class<? extends Map>, Byte> mapTypes = new HashMap<>();
+
+    /** */
+    private final ConcurrentMap<Integer, BinaryIdMapper> mappers = new ConcurrentHashMap8<>(0);
+
+    /** Affinity key field names. */
+    private final ConcurrentMap<Integer, String> affKeyFieldNames = new ConcurrentHashMap8<>(0);
+
+    /** */
+    private final Map<String, BinaryIdMapper> typeMappers = new ConcurrentHashMap8<>(0);
+
+    /** */
+    private BinaryMetadataHandler metaHnd;
+
+    /** Actual marshaller. */
+    private BinaryMarshaller marsh;
+
+    /** */
+    private MarshallerContext marshCtx;
+
+    /** */
+    private String gridName;
+
+    /** */
+    private IgniteConfiguration igniteCfg;
+
+    /** */
+    private final OptimizedMarshaller optmMarsh = new OptimizedMarshaller();
+
+    /** Compact footer flag. */
+    private boolean compactFooter;
+
+    /** Object schemas. */
+    private volatile Map<Integer, BinarySchemaRegistry> schemas;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public BinaryContext() {
+        // No-op.
+    }
+
+    /**
+     * @param metaHnd Meta data handler.
+     * @param igniteCfg Ignite configuration.
+     */
+    public BinaryContext(BinaryMetadataHandler metaHnd, IgniteConfiguration igniteCfg) {
+        assert metaHnd != null;
+        assert igniteCfg != null;
+
+        this.metaHnd = metaHnd;
+        this.igniteCfg = igniteCfg;
+
+        gridName = igniteCfg.getGridName();
+
+        colTypes.put(ArrayList.class, GridBinaryMarshaller.ARR_LIST);
+        colTypes.put(LinkedList.class, GridBinaryMarshaller.LINKED_LIST);
+        colTypes.put(HashSet.class, GridBinaryMarshaller.HASH_SET);
+        colTypes.put(LinkedHashSet.class, GridBinaryMarshaller.LINKED_HASH_SET);
+
+        mapTypes.put(HashMap.class, GridBinaryMarshaller.HASH_MAP);
+        mapTypes.put(LinkedHashMap.class, GridBinaryMarshaller.LINKED_HASH_MAP);
+
+        // IDs range from [0..200] is used by Java SDK API and GridGain legacy API
+
+        registerPredefinedType(Byte.class, GridBinaryMarshaller.BYTE);
+        registerPredefinedType(Boolean.class, GridBinaryMarshaller.BOOLEAN);
+        registerPredefinedType(Short.class, GridBinaryMarshaller.SHORT);
+        registerPredefinedType(Character.class, GridBinaryMarshaller.CHAR);
+        registerPredefinedType(Integer.class, GridBinaryMarshaller.INT);
+        registerPredefinedType(Long.class, GridBinaryMarshaller.LONG);
+        registerPredefinedType(Float.class, GridBinaryMarshaller.FLOAT);
+        registerPredefinedType(Double.class, GridBinaryMarshaller.DOUBLE);
+        registerPredefinedType(String.class, GridBinaryMarshaller.STRING);
+        registerPredefinedType(BigDecimal.class, GridBinaryMarshaller.DECIMAL);
+        registerPredefinedType(Date.class, GridBinaryMarshaller.DATE);
+        registerPredefinedType(Timestamp.class, GridBinaryMarshaller.TIMESTAMP);
+        registerPredefinedType(UUID.class, GridBinaryMarshaller.UUID);
+
+        registerPredefinedType(byte[].class, GridBinaryMarshaller.BYTE_ARR);
+        registerPredefinedType(short[].class, GridBinaryMarshaller.SHORT_ARR);
+        registerPredefinedType(int[].class, GridBinaryMarshaller.INT_ARR);
+        registerPredefinedType(long[].class, GridBinaryMarshaller.LONG_ARR);
+        registerPredefinedType(float[].class, GridBinaryMarshaller.FLOAT_ARR);
+        registerPredefinedType(double[].class, GridBinaryMarshaller.DOUBLE_ARR);
+        registerPredefinedType(char[].class, GridBinaryMarshaller.CHAR_ARR);
+        registerPredefinedType(boolean[].class, GridBinaryMarshaller.BOOLEAN_ARR);
+        registerPredefinedType(BigDecimal[].class, GridBinaryMarshaller.DECIMAL_ARR);
+        registerPredefinedType(String[].class, GridBinaryMarshaller.STRING_ARR);
+        registerPredefinedType(UUID[].class, GridBinaryMarshaller.UUID_ARR);
+        registerPredefinedType(Date[].class, GridBinaryMarshaller.DATE_ARR);
+        registerPredefinedType(Timestamp[].class, GridBinaryMarshaller.TIMESTAMP_ARR);
+        registerPredefinedType(Object[].class, GridBinaryMarshaller.OBJ_ARR);
+
+        registerPredefinedType(ArrayList.class, 0);
+        registerPredefinedType(LinkedList.class, 0);
+        registerPredefinedType(HashSet.class, 0);
+        registerPredefinedType(LinkedHashSet.class, 0);
+
+        registerPredefinedType(HashMap.class, 0);
+        registerPredefinedType(LinkedHashMap.class, 0);
+
+        registerPredefinedType(GridMapEntry.class, 60);
+        registerPredefinedType(IgniteBiTuple.class, 61);
+        registerPredefinedType(T2.class, 62);
+
+        // IDs range [200..1000] is used by Ignite internal APIs.
+    }
+
+    /**
+     * @return Marshaller.
+     */
+    public BinaryMarshaller marshaller() {
+        return marsh;
+    }
+
+    /**
+     * @return Ignite configuration.
+     */
+    public IgniteConfiguration configuration(){
+        return igniteCfg;
+    }
+
+    /**
+     * @param marsh Portable marshaller.
+     * @param cfg Configuration.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void configure(BinaryMarshaller marsh, IgniteConfiguration cfg) throws BinaryObjectException {
+        if (marsh == null)
+            return;
+
+        this.marsh = marsh;
+
+        marshCtx = marsh.getContext();
+
+        BinaryConfiguration binaryCfg = cfg.getBinaryConfiguration();
+
+        if (binaryCfg == null)
+            binaryCfg = new BinaryConfiguration();
+
+        assert marshCtx != null;
+
+        optmMarsh.setContext(marshCtx);
+
+        configure(
+            binaryCfg.getIdMapper(),
+            binaryCfg.getSerializer(),
+            binaryCfg.getTypeConfigurations()
+        );
+
+        compactFooter = binaryCfg.isCompactFooter();
+    }
+
+    /**
+     * @param globalIdMapper ID mapper.
+     * @param globalSerializer Serializer.
+     * @param typeCfgs Type configurations.
+     * @throws BinaryObjectException In case of error.
+     */
+    private void configure(
+        BinaryIdMapper globalIdMapper,
+        BinarySerializer globalSerializer,
+        Collection<BinaryTypeConfiguration> typeCfgs
+    ) throws BinaryObjectException {
+        TypeDescriptors descs = new TypeDescriptors();
+
+        Map<String, String> affFields = new HashMap<>();
+
+        if (!F.isEmpty(igniteCfg.getCacheKeyConfiguration())) {
+            for (CacheKeyConfiguration keyCfg : igniteCfg.getCacheKeyConfiguration())
+                affFields.put(keyCfg.getTypeName(), keyCfg.getAffinityKeyFieldName());
+        }
+
+        if (typeCfgs != null) {
+            for (BinaryTypeConfiguration typeCfg : typeCfgs) {
+                String clsName = typeCfg.getTypeName();
+
+                if (clsName == null)
+                    throw new BinaryObjectException("Class name is required for portable type configuration.");
+
+                BinaryIdMapper idMapper = globalIdMapper;
+
+                if (typeCfg.getIdMapper() != null)
+                    idMapper = typeCfg.getIdMapper();
+
+                idMapper = BinaryInternalIdMapper.create(idMapper);
+
+                BinarySerializer serializer = globalSerializer;
+
+                if (typeCfg.getSerializer() != null)
+                    serializer = typeCfg.getSerializer();
+
+                if (clsName.endsWith(".*")) {
+                    String pkgName = clsName.substring(0, clsName.length() - 2);
+
+                    for (String clsName0 : classesInPackage(pkgName))
+                        descs.add(clsName0, idMapper, serializer, affFields.get(clsName0),
+                            typeCfg.isEnum(), true);
+                }
+                else
+                    descs.add(clsName, idMapper, serializer, affFields.get(clsName),
+                        typeCfg.isEnum(), false);
+            }
+        }
+
+        for (TypeDescriptor desc : descs.descriptors())
+            registerUserType(desc.clsName, desc.idMapper, desc.serializer, desc.affKeyFieldName, desc.isEnum);
+
+        BinaryInternalIdMapper dfltMapper = BinaryInternalIdMapper.create(globalIdMapper);
+
+        // Put affinity field names for unconfigured types.
+        for (Map.Entry<String, String> entry : affFields.entrySet()) {
+            String typeName = entry.getKey();
+
+            int typeId = dfltMapper.typeId(typeName);
+
+            affKeyFieldNames.putIfAbsent(typeId, entry.getValue());
+        }
+
+        addSystemClassAffinityKey(CollocatedSetItemKey.class);
+        addSystemClassAffinityKey(CollocatedQueueItemKey.class);
+    }
+
+    /**
+     * @param cls Class.
+     */
+    private void addSystemClassAffinityKey(Class<?> cls) {
+        String fieldName = affinityFieldName(cls);
+
+        assert fieldName != null : cls;
+
+        affKeyFieldNames.putIfAbsent(cls.getName().hashCode(), affinityFieldName(cls));
+    }
+
+    /**
+     * @param pkgName Package name.
+     * @return Class names.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private static Iterable<String> classesInPackage(String pkgName) {
+        assert pkgName != null;
+
+        Collection<String> clsNames = new ArrayList<>();
+
+        ClassLoader ldr = U.gridClassLoader();
+
+        if (ldr instanceof URLClassLoader) {
+            String pkgPath = pkgName.replaceAll("\\.", "/");
+
+            URL[] urls = ((URLClassLoader)ldr).getURLs();
+
+            for (URL url : urls) {
+                String proto = url.getProtocol().toLowerCase();
+
+                if ("file".equals(proto)) {
+                    try {
+                        File cpElement = new File(url.toURI());
+
+                        if (cpElement.isDirectory()) {
+                            File pkgDir = new File(cpElement, pkgPath);
+
+                            if (pkgDir.isDirectory()) {
+                                for (File file : pkgDir.listFiles()) {
+                                    String fileName = file.getName();
+
+                                    if (file.isFile() && fileName.toLowerCase().endsWith(".class"))
+                                        clsNames.add(pkgName + '.' + fileName.substring(0, fileName.length() - 6));
+                                }
+                            }
+                        }
+                        else if (cpElement.isFile()) {
+                            try {
+                                JarFile jar = new JarFile(cpElement);
+
+                                Enumeration<JarEntry> entries = jar.entries();
+
+                                while (entries.hasMoreElements()) {
+                                    String entry = entries.nextElement().getName();
+
+                                    if (entry.startsWith(pkgPath) && entry.endsWith(".class")) {
+                                        String clsName = entry.substring(pkgPath.length() + 1, entry.length() - 6);
+
+                                        if (!clsName.contains("/") && !clsName.contains("\\"))
+                                            clsNames.add(pkgName + '.' + clsName);
+                                    }
+                                }
+                            }
+                            catch (IOException ignored) {
+                                // No-op.
+                            }
+                        }
+                    }
+                    catch (URISyntaxException ignored) {
+                        // No-op.
+                    }
+                }
+            }
+        }
+
+        return clsNames;
+    }
+
+    /**
+     * @param cls Class.
+     * @return Class descriptor.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BinaryClassDescriptor descriptorForClass(Class<?> cls, boolean deserialize)
+        throws BinaryObjectException {
+        assert cls != null;
+
+        BinaryClassDescriptor desc = descByCls.get(cls);
+
+        if (desc == null || !desc.registered())
+            desc = registerClassDescriptor(cls, deserialize);
+
+        return desc;
+    }
+
+    /**
+     * @param userType User type or not.
+     * @param typeId Type ID.
+     * @param ldr Class loader.
+     * @return Class descriptor.
+     */
+    public BinaryClassDescriptor descriptorForTypeId(
+        boolean userType,
+        int typeId,
+        ClassLoader ldr,
+        boolean deserialize
+    ) {
+        assert typeId != GridBinaryMarshaller.UNREGISTERED_TYPE_ID;
+
+        //TODO: As a workaround for IGNITE-1358 we always check the predefined map before without checking 'userType'
+        BinaryClassDescriptor desc = predefinedTypes.get(typeId);
+
+        if (desc != null)
+            return desc;
+
+        if (ldr == null)
+            ldr = dfltLdr;
+
+        // If the type hasn't been loaded by default class loader then we mustn't return the descriptor from here
+        // giving a chance to a custom class loader to reload type's class.
+        if (userType && ldr.equals(dfltLdr)) {
+            desc = userTypes.get(typeId);
+
+            if (desc != null)
+                return desc;
+        }
+
+        Class cls;
+
+        try {
+            cls = marshCtx.getClass(typeId, ldr);
+
+            desc = descByCls.get(cls);
+        }
+        catch (ClassNotFoundException e) {
+            // Class might have been loaded by default class loader.
+            if (userType && !ldr.equals(dfltLdr) && (desc = descriptorForTypeId(true, typeId, dfltLdr, deserialize)) != null)
+                return desc;
+
+            throw new BinaryInvalidTypeException(e);
+        }
+        catch (IgniteCheckedException e) {
+            // Class might have been loaded by default class loader.
+            if (userType && !ldr.equals(dfltLdr) && (desc = descriptorForTypeId(true, typeId, dfltLdr, deserialize)) != null)
+                return desc;
+
+            throw new BinaryObjectException("Failed resolve class for ID: " + typeId, e);
+        }
+
+        if (desc == null) {
+            desc = registerClassDescriptor(cls, deserialize);
+
+            assert desc.typeId() == typeId;
+        }
+
+        return desc;
+    }
+
+    /**
+     * Creates and registers {@link BinaryClassDescriptor} for the given {@code class}.
+     *
+     * @param cls Class.
+     * @return Class descriptor.
+     */
+    private BinaryClassDescriptor registerClassDescriptor(Class<?> cls, boolean deserialize) {
+        BinaryClassDescriptor desc;
+
+        String clsName = cls.getName();
+
+        if (marshCtx.isSystemType(clsName)) {
+            desc = new BinaryClassDescriptor(this,
+                cls,
+                false,
+                clsName.hashCode(),
+                clsName,
+                null,
+                BinaryInternalIdMapper.defaultInstance(),
+                null,
+                false,
+                true, /* registered */
+                false /* predefined */
+            );
+
+            BinaryClassDescriptor old = descByCls.putIfAbsent(cls, desc);
+
+            if (old != null)
+                desc = old;
+        }
+        else
+            desc = registerUserClassDescriptor(cls, deserialize);
+
+        return desc;
+    }
+
+    /**
+     * Creates and registers {@link BinaryClassDescriptor} for the given user {@code class}.
+     *
+     * @param cls Class.
+     * @return Class descriptor.
+     */
+    private BinaryClassDescriptor registerUserClassDescriptor(Class<?> cls, boolean deserialize) {
+        boolean registered;
+
+        String typeName = typeName(cls.getName());
+
+        BinaryIdMapper idMapper = userTypeIdMapper(typeName);
+
+        int typeId = idMapper.typeId(typeName);
+
+        try {
+            registered = marshCtx.registerClass(typeId, cls);
+        }
+        catch (IgniteCheckedException e) {
+            throw new BinaryObjectException("Failed to register class.", e);
+        }
+
+        String affFieldName = affinityFieldName(cls);
+
+        BinaryClassDescriptor desc = new BinaryClassDescriptor(this,
+            cls,
+            true,
+            typeId,
+            typeName,
+            affFieldName,
+            idMapper,
+            null,
+            true,
+            registered,
+            false /* predefined */
+        );
+
+        if (!deserialize) {
+            Collection<BinarySchema> schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null;
+
+            metaHnd.addMeta(typeId,
+                new BinaryMetadata(typeId, typeName, desc.fieldsMeta(), affFieldName, schemas, desc.isEnum()).wrap(this));
+        }
+
+        // perform put() instead of putIfAbsent() because "registered" flag might have been changed or class loader
+        // might have reloaded described class.
+        if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr))
+            userTypes.put(typeId, desc);
+
+        descByCls.put(cls, desc);
+
+        mappers.putIfAbsent(typeId, idMapper);
+
+        return desc;
+    }
+
+    /**
+     * @param cls Collection class.
+     * @return Collection type ID.
+     */
+    public byte collectionType(Class<? extends Collection> cls) {
+        assert cls != null;
+
+        Byte type = colTypes.get(cls);
+
+        if (type != null)
+            return type;
+
+        return Set.class.isAssignableFrom(cls) ? GridBinaryMarshaller.USER_SET : GridBinaryMarshaller.USER_COL;
+    }
+
+    /**
+     * @param cls Map class.
+     * @return Map type ID.
+     */
+    public byte mapType(Class<? extends Map> cls) {
+        assert cls != null;
+
+        Byte type = mapTypes.get(cls);
+
+        return type != null ? type : GridBinaryMarshaller.USER_COL;
+    }
+
+    /**
+     * @param typeName Type name.
+     * @return Type ID.
+     */
+    public int typeId(String typeName) {
+        String typeName0 = typeName(typeName);
+
+        Integer id = predefinedTypeNames.get(typeName0);
+
+        if (id != null)
+            return id;
+
+        if (marshCtx.isSystemType(typeName))
+            return typeName.hashCode();
+
+        return userTypeIdMapper(typeName0).typeId(typeName0);
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName) {
+        return userTypeIdMapper(typeId).fieldId(typeId, fieldName);
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @return Instance of ID mapper.
+     */
+    public BinaryIdMapper userTypeIdMapper(int typeId) {
+        BinaryIdMapper idMapper = mappers.get(typeId);
+
+        return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance();
+    }
+
+    /**
+     * @param typeName Type name.
+     * @return Instance of ID mapper.
+     */
+    private BinaryIdMapper userTypeIdMapper(String typeName) {
+        BinaryIdMapper idMapper = typeMappers.get(typeName);
+
+        return idMapper != null ? idMapper : BinaryInternalIdMapper.defaultInstance();
+    }
+
+    /**
+     * @param cls Class to get affinity field for.
+     * @return Affinity field name or {@code null} if field name was not found.
+     */
+    private String affinityFieldName(Class cls) {
+        for (; cls != Object.class && cls != null; cls = cls.getSuperclass()) {
+            for (Field f : cls.getDeclaredFields()) {
+                if (f.getAnnotation(AffinityKeyMapped.class) != null)
+                    return f.getName();
+            }
+        }
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, igniteCfg.getGridName());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        gridName = U.readString(in);
+    }
+
+    /**
+     * @return Portable context.
+     * @throws ObjectStreamException In case of error.
+     */
+    protected Object readResolve() throws ObjectStreamException {
+        try {
+            IgniteKernal g = IgnitionEx.gridx(gridName);
+
+            if (g == null)
+                throw new IllegalStateException("Failed to find grid for name: " + gridName);
+
+            return ((CacheObjectBinaryProcessorImpl)g.context().cacheObjects()).portableContext();
+        }
+        catch (IllegalStateException e) {
+            throw U.withCause(new InvalidObjectException(e.getMessage()), e);
+        }
+    }
+
+    /**
+     * @param cls Class.
+     * @param id Type ID.
+     * @return GridPortableClassDescriptor.
+     */
+    public BinaryClassDescriptor registerPredefinedType(Class<?> cls, int id) {
+        String typeName = typeName(cls.getName());
+
+        BinaryClassDescriptor desc = new BinaryClassDescriptor(
+            this,
+            cls,
+            false,
+            id,
+            typeName,
+            null,
+            BinaryInternalIdMapper.defaultInstance(),
+            null,
+            false,
+            true, /* registered */
+            true /* predefined */
+        );
+
+        predefinedTypeNames.put(typeName, id);
+        predefinedTypes.put(id, desc);
+
+        descByCls.put(cls, desc);
+
+        return desc;
+    }
+
+    /**
+     * @param clsName Class name.
+     * @param idMapper ID mapper.
+     * @param serializer Serializer.
+     * @param affKeyFieldName Affinity key field name.
+     * @param isEnum If enum.
+     * @throws BinaryObjectException In case of error.
+     */
+    @SuppressWarnings("ErrorNotRethrown")
+    public void registerUserType(String clsName,
+        BinaryIdMapper idMapper,
+        @Nullable BinarySerializer serializer,
+        @Nullable String affKeyFieldName,
+        boolean isEnum)
+        throws BinaryObjectException {
+        assert idMapper != null;
+
+        Class<?> cls = null;
+
+        try {
+            cls = Class.forName(clsName);
+        }
+        catch (ClassNotFoundException | NoClassDefFoundError ignored) {
+            // No-op.
+        }
+
+        String typeName = typeName(clsName);
+
+        int id = idMapper.typeId(typeName);
+
+        //Workaround for IGNITE-1358
+        if (predefinedTypes.get(id) != null)
+            throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+
+        if (mappers.put(id, idMapper) != null)
+            throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+
+        if (affKeyFieldName != null) {
+            if (affKeyFieldNames.put(id, affKeyFieldName) != null)
+                throw new BinaryObjectException("Duplicate type ID [clsName=" + clsName + ", id=" + id + ']');
+        }
+
+        typeMappers.put(typeName, idMapper);
+
+        Map<String, Integer> fieldsMeta = null;
+        Collection<BinarySchema> schemas = null;
+
+        if (cls != null) {
+            BinaryClassDescriptor desc = new BinaryClassDescriptor(
+                this,
+                cls,
+                true,
+                id,
+                typeName,
+                affKeyFieldName,
+                idMapper,
+                serializer,
+                true,
+                true, /* registered */
+                false /* predefined */
+            );
+
+            fieldsMeta = desc.fieldsMeta();
+            schemas = desc.schema() != null ? Collections.singleton(desc.schema()) : null;
+
+            if (IgniteUtils.detectClassLoader(cls).equals(dfltLdr))
+                userTypes.put(id, desc);
+
+            descByCls.put(cls, desc);
+        }
+
+        metaHnd.addMeta(id, new BinaryMetadata(id, typeName, fieldsMeta, affKeyFieldName, schemas, isEnum).wrap(this));
+    }
+
+    /**
+     * Create binary field.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Binary field.
+     */
+    public BinaryFieldImpl createField(int typeId, String fieldName) {
+        BinarySchemaRegistry schemaReg = schemaRegistry(typeId);
+
+        int fieldId = userTypeIdMapper(typeId).fieldId(typeId, fieldName);
+
+        return new BinaryFieldImpl(typeId, schemaReg, fieldName, fieldId);
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @return Meta data.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public BinaryType metadata(int typeId) throws BinaryObjectException {
+        return metaHnd != null ? metaHnd.metadata(typeId) : null;
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @return Affinity key field name.
+     */
+    public String affinityKeyFieldName(int typeId) {
+        return affKeyFieldNames.get(typeId);
+    }
+
+    /**
+     * @param typeId Type ID.
+     * @param meta Meta data.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void updateMetadata(int typeId, BinaryMetadata meta) throws BinaryObjectException {
+        metaHnd.addMeta(typeId, meta.wrap(this));
+    }
+
+    /**
+     * @return Whether field IDs should be skipped in footer or not.
+     */
+    public boolean isCompactFooter() {
+        return compactFooter;
+    }
+
+    /**
+     * Get schema registry for type ID.
+     *
+     * @param typeId Type ID.
+     * @return Schema registry for type ID.
+     */
+    public BinarySchemaRegistry schemaRegistry(int typeId) {
+        Map<Integer, BinarySchemaRegistry> schemas0 = schemas;
+
+        if (schemas0 == null) {
+            synchronized (this) {
+                schemas0 = schemas;
+
+                if (schemas0 == null) {
+                    schemas0 = new HashMap<>();
+
+                    BinarySchemaRegistry reg = new BinarySchemaRegistry();
+
+                    schemas0.put(typeId, reg);
+
+                    schemas = schemas0;
+
+                    return reg;
+                }
+            }
+        }
+
+        BinarySchemaRegistry reg = schemas0.get(typeId);
+
+        if (reg == null) {
+            synchronized (this) {
+                reg = schemas.get(typeId);
+
+                if (reg == null) {
+                    reg = new BinarySchemaRegistry();
+
+                    schemas0 = new HashMap<>(schemas);
+
+                    schemas0.put(typeId, reg);
+
+                    schemas = schemas0;
+                }
+            }
+        }
+
+        return reg;
+    }
+
+    /**
+     * Returns instance of {@link OptimizedMarshaller}.
+     *
+     * @return Optimized marshaller.
+     */
+    OptimizedMarshaller optimizedMarsh() {
+        return optmMarsh;
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return Type name.
+     */
+    @SuppressWarnings("ResultOfMethodCallIgnored")
+    public static String typeName(String clsName) {
+        assert clsName != null;
+
+        int idx = clsName.lastIndexOf('$');
+
+        if (idx == clsName.length() - 1)
+            // This is a regular (not inner) class name that ends with '$'. Common use case for Scala classes.
+            idx = -1;
+        else if (idx >= 0) {
+            String typeName = clsName.substring(idx + 1);
+
+            try {
+                Integer.parseInt(typeName);
+
+                // This is an anonymous class. Don't cut off enclosing class name for it.
+                idx = -1;
+            }
+            catch (NumberFormatException ignore) {
+                // This is a lambda class.
+                if (clsName.indexOf("$$Lambda$") > 0)
+                    idx = -1;
+                else
+                    return typeName;
+            }
+        }
+
+        if (idx < 0)
+            idx = clsName.lastIndexOf('.');
+
+        return idx >= 0 ? clsName.substring(idx + 1) : clsName;
+    }
+
+    /**
+     * Undeployment callback invoked when class loader is being undeployed.
+     *
+     * Some marshallers may want to clean their internal state that uses the undeployed class loader somehow.
+     *
+     * @param ldr Class loader being undeployed.
+     */
+    public void onUndeploy(ClassLoader ldr) {
+        for (Class<?> cls : descByCls.keySet()) {
+            if (ldr.equals(cls.getClassLoader()))
+                descByCls.remove(cls);
+        }
+
+        U.clearClassCache(ldr);
+    }
+
+    /**
+     * Type descriptors.
+     */
+    private static class TypeDescriptors {
+        /** Descriptors map. */
+        private final Map<String, TypeDescriptor> descs = new LinkedHashMap<>();
+
+        /**
+         * Add type descriptor.
+         *
+         * @param clsName Class name.
+         * @param idMapper ID mapper.
+         * @param serializer Serializer.
+         * @param affKeyFieldName Affinity key field name.
+         * @param isEnum Enum flag.
+         * @param canOverride Whether this descriptor can be override.
+         * @throws BinaryObjectException If failed.
+         */
+        private void add(String clsName,
+            BinaryIdMapper idMapper,
+            BinarySerializer serializer,
+            String affKeyFieldName,
+            boolean isEnum,
+            boolean canOverride)
+            throws BinaryObjectException {
+            TypeDescriptor desc = new TypeDescriptor(clsName,
+                idMapper,
+                serializer,
+                affKeyFieldName,
+                isEnum,
+                canOverride);
+
+            TypeDescriptor oldDesc = descs.get(clsName);
+
+            if (oldDesc == null)
+                descs.put(clsName, desc);
+            else
+                oldDesc.override(desc);
+        }
+
+        /**
+         * Get all collected descriptors.
+         *
+         * @return Descriptors.
+         */
+        private Iterable<TypeDescriptor> descriptors() {
+            return descs.values();
+        }
+    }
+
+    /**
+     * Type descriptor.
+     */
+    private static class TypeDescriptor {
+        /** Class name. */
+        private final String clsName;
+
+        /** ID mapper. */
+        private BinaryIdMapper idMapper;
+
+        /** Serializer. */
+        private BinarySerializer serializer;
+
+        /** Affinity key field name. */
+        private String affKeyFieldName;
+
+        /** Enum flag. */
+        private boolean isEnum;
+
+        /** Whether this descriptor can be override. */
+        private boolean canOverride;
+
+        /**
+         * Constructor.
+         *
+         * @param clsName Class name.
+         * @param idMapper ID mapper.
+         * @param serializer Serializer.
+         * @param affKeyFieldName Affinity key field name.
+         * @param isEnum Enum type.
+         * @param canOverride Whether this descriptor can be override.
+         */
+        private TypeDescriptor(String clsName, BinaryIdMapper idMapper, BinarySerializer serializer,
+            String affKeyFieldName, boolean isEnum, boolean canOverride) {
+            this.clsName = clsName;
+            this.idMapper = idMapper;
+            this.serializer = serializer;
+            this.affKeyFieldName = affKeyFieldName;
+            this.isEnum = isEnum;
+            this.canOverride = canOverride;
+        }
+
+        /**
+         * Override portable class descriptor.
+         *
+         * @param other Other descriptor.
+         * @throws BinaryObjectException If failed.
+         */
+        private void override(TypeDescriptor other) throws BinaryObjectException {
+            assert clsName.equals(other.clsName);
+
+            if (canOverride) {
+                idMapper = other.idMapper;
+                serializer = other.serializer;
+                affKeyFieldName = other.affKeyFieldName;
+                canOverride = other.canOverride;
+            }
+            else if (!other.canOverride)
+                throw new BinaryObjectException("Duplicate explicit class definition in configuration: " + clsName);
+        }
+    }
+
+    /**
+     * Type id wrapper.
+     */
+    static class Type {
+        /** Type id */
+        private final int id;
+
+        /** Whether the following type is registered in a cache or not */
+        private final boolean registered;
+
+        /**
+         * @param id Id.
+         * @param registered Registered.
+         */
+        public Type(int id, boolean registered) {
+            this.id = id;
+            this.registered = registered;
+        }
+
+        /**
+         * @return Type ID.
+         */
+        public int id() {
+            return id;
+        }
+
+        /**
+         * @return Registered flag value.
+         */
+        public boolean registered() {
+            return registered;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 15e42e3..3321170 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -45,7 +45,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** Context. */
     @GridDirectTransient
-    private PortableContext ctx;
+    private BinaryContext ctx;
 
     /** Type ID. */
     private int typeId;
@@ -71,7 +71,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
      * @param clsName Class name.
      * @param ord Ordinal.
      */
-    public BinaryEnumObjectImpl(PortableContext ctx, int typeId, @Nullable String clsName, int ord) {
+    public BinaryEnumObjectImpl(BinaryContext ctx, int typeId, @Nullable String clsName, int ord) {
         assert ctx != null;
 
         this.ctx = ctx;
@@ -110,7 +110,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <T> T deserialize() throws BinaryObjectException {
-        Class cls = PortableUtils.resolveClass(ctx, typeId, clsName, null, true);
+        Class cls = BinaryUtils.resolveClass(ctx, typeId, clsName, null, true);
 
         return BinaryEnumCache.get(cls, ord);
     }
@@ -167,7 +167,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
             return type.typeName() + "[ordinal=" + ord  + ']';
         }
         else {
-            if (typeId == GridPortableMarshaller.UNREGISTERED_TYPE_ID)
+            if (typeId == GridBinaryMarshaller.UNREGISTERED_TYPE_ID)
                 return "BinaryEnum[clsName=" + clsName + ", ordinal=" + ord + ']';
             else
                 return "BinaryEnum[typeId=" + typeId + ", ordinal=" + ord + ']';
@@ -185,7 +185,7 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        ctx = (PortableContext)in.readObject();
+        ctx = (BinaryContext)in.readObject();
 
         typeId = in.readInt();
         clsName = (String)in.readObject();

http://git-wip-us.apache.org/repos/asf/ignite/blob/71ad9cea/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
index 962805d..8cf1a11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryFieldAccessor.java
@@ -48,7 +48,7 @@ public abstract class BinaryFieldAccessor {
      * @return Accessor.
      */
     public static BinaryFieldAccessor create(Field field, int id) {
-        BinaryWriteMode mode = PortableUtils.mode(field.getType());
+        BinaryWriteMode mode = BinaryUtils.mode(field.getType());
 
         switch (mode) {
             case P_BYTE:
@@ -849,7 +849,7 @@ public abstract class BinaryFieldAccessor {
          */
         protected BinaryWriteMode mode(Object val) {
             return dynamic ?
-                val == null ? BinaryWriteMode.OBJECT : PortableUtils.mode(val.getClass()) :
+                val == null ? BinaryWriteMode.OBJECT : BinaryUtils.mode(val.getClass()) :
                 mode;
         }
     }