You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/01/19 12:05:41 UTC

[1/5] ignite git commit: IGNITE-4157 Use discovery custom messages instead of marshaller cache - Fixes #1271.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 454b9769e -> 4cd332b78


http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
index d1fd4e7..f8ea38a 100644
--- a/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/thread/IgniteThreadPoolSizeTest.java
@@ -95,14 +95,6 @@ public class IgniteThreadPoolSizeTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    @SuppressWarnings("deprecated")
-    public void testMarshallerCachePoolSize() throws Exception {
-        testWrongPoolSize(configuration().setMarshallerCachePoolSize(WRONG_VALUE));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testConnectorThreadPoolSize() throws Exception {
         final IgniteConfiguration cfg = configuration();
 


[3/5] ignite git commit: IGNITE-4157 Use discovery custom messages instead of marshaller cache - Fixes #1271.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingItem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingItem.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingItem.java
new file mode 100644
index 0000000..1cff458
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingItem.java
@@ -0,0 +1,99 @@
+/*
+ * 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.processors.marshaller;
+
+import java.io.Serializable;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *  Used to exchange mapping information on new mapping added or missing mapping requested flows.
+ *  See {@link GridMarshallerMappingProcessor} javadoc for more information.
+ */
+public final class MarshallerMappingItem implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final byte platformId;
+
+    /** */
+    private final int typeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * Class name may be null when instance is created to request missing mapping from cluster.
+     *
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     * @param clsName Class name. May be null in case when the item is created to request missing mapping from grid.
+     */
+    public MarshallerMappingItem(byte platformId, int typeId, @Nullable String clsName) {
+        this.platformId = platformId;
+        this.typeId = typeId;
+        this.clsName = clsName;
+    }
+
+    /** */
+    public int typeId() {
+        return typeId;
+    }
+
+    /** */
+    public byte platformId() {
+        return platformId;
+    }
+
+    /** */
+    public String className() {
+        return clsName;
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public void className(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (obj == this)
+            return true;
+
+        if (!(obj instanceof MarshallerMappingItem))
+            return false;
+
+        MarshallerMappingItem that = (MarshallerMappingItem) obj;
+
+        return platformId == that.platformId
+                && typeId == that.typeId
+                && (clsName != null ? clsName.equals(that.clsName) : that.clsName == null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return 31 * 31 * ((int) platformId) + 31 * typeId + (clsName != null ? clsName.hashCode() : 0) ;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "[platformId: " + platformId + ", typeId:" + typeId + ", clsName: " + clsName + "]";
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingTransport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingTransport.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingTransport.java
new file mode 100644
index 0000000..b974882
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MarshallerMappingTransport.java
@@ -0,0 +1,212 @@
+/*
+ * 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.processors.marshaller;
+
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Provides capabilities of sending custom discovery events to propose new mapping
+ * or request missing mapping to {@link MarshallerContextImpl}.
+ *
+ * For more information about particular events see documentation of {@link GridMarshallerMappingProcessor}.
+ */
+public final class MarshallerMappingTransport {
+    /** */
+    private final GridKernalContext ctx;
+
+    /** */
+    private final GridDiscoveryManager discoMgr;
+
+    /** */
+    private final ConcurrentMap<MarshallerMappingItem, GridFutureAdapter<MappingExchangeResult>> mappingExchSyncMap;
+
+    /** */
+    private final ConcurrentMap<MarshallerMappingItem, ClientRequestFuture> clientReqSyncMap;
+
+    /** */
+    private volatile boolean stopping;
+
+    /**
+     * @param ctx Context.
+     * @param mappingExchSyncMap Mapping exch sync map.
+     * @param clientReqSyncMap Client request sync map.
+     */
+    MarshallerMappingTransport(
+            GridKernalContext ctx,
+            ConcurrentMap<MarshallerMappingItem, GridFutureAdapter<MappingExchangeResult>> mappingExchSyncMap,
+            ConcurrentMap<MarshallerMappingItem, ClientRequestFuture> clientReqSyncMap
+    ) {
+        this.ctx = ctx;
+        discoMgr = ctx.discovery();
+        this.mappingExchSyncMap = mappingExchSyncMap;
+        this.clientReqSyncMap = clientReqSyncMap;
+
+        stopping = false;
+    }
+
+    /**
+     * @param item Item.
+     * @param cache Cache.
+     */
+    public GridFutureAdapter<MappingExchangeResult> awaitMappingAcceptance(
+            MarshallerMappingItem item, ConcurrentMap<Integer,
+            MappedName> cache
+    ) {
+        GridFutureAdapter<MappingExchangeResult> fut = new MappingExchangeResultFuture(item);
+
+        GridFutureAdapter<MappingExchangeResult> oldFut = mappingExchSyncMap.putIfAbsent(item, fut);
+
+        if (oldFut != null)
+            return oldFut;
+
+        MappedName mappedName = cache.get(item.typeId());
+
+        assert mappedName != null;
+
+        //double check whether mapping is accepted, first check was in MarshallerContextImpl::registerClassName
+        if (mappedName.accepted())
+            fut.onDone(MappingExchangeResult.createSuccessfulResult(mappedName.className()));
+
+        return fut;
+    }
+
+    /**
+     * @param item Item.
+     * @param cache Cache.
+     */
+    public GridFutureAdapter<MappingExchangeResult> proposeMapping(MarshallerMappingItem item, ConcurrentMap<Integer, MappedName> cache) throws IgniteCheckedException {
+        GridFutureAdapter<MappingExchangeResult> fut = new MappingExchangeResultFuture(item);
+
+        GridFutureAdapter<MappingExchangeResult> oldFut = mappingExchSyncMap.putIfAbsent(item, fut);
+
+        if (oldFut != null)
+            return oldFut;
+        else {
+            //double check, first check was in caller: MarshallerContextImpl::registerClassName
+            MappedName mapping = cache.get(item.typeId());
+
+            if (mapping != null) {
+                String mappedClsName = mapping.className();
+
+                if (!mappedClsName.equals(item.className()))
+                    fut.onDone(MappingExchangeResult.createFailureResult(duplicateMappingException(item, mappedClsName)));
+                else if (mapping.accepted())
+                    fut.onDone(MappingExchangeResult.createSuccessfulResult(mappedClsName));
+                else if (stopping)
+                    fut.onDone(MappingExchangeResult.createExchangeDisabledResult());
+
+                return fut;
+            }
+        }
+
+        DiscoveryCustomMessage msg = new MappingProposedMessage(item, discoMgr.localNode().id());
+        discoMgr.sendCustomEvent(msg);
+
+        return fut;
+    }
+
+    /**
+     * @param item Item.
+     * @param cache Cache.
+     */
+    public GridFutureAdapter<MappingExchangeResult> requestMapping(
+            MarshallerMappingItem item,
+            ConcurrentMap<Integer, MappedName> cache
+    ) {
+        ClientRequestFuture newFut = new ClientRequestFuture(ctx, item, clientReqSyncMap);
+
+        ClientRequestFuture oldFut = clientReqSyncMap.putIfAbsent(item, newFut);
+
+        if (oldFut != null)
+            return oldFut;
+
+        MappedName mappedName = cache.get(item.typeId());
+
+        if (mappedName != null) {
+            newFut.onDone(MappingExchangeResult.createSuccessfulResult(mappedName.className()));
+
+            return newFut;
+        }
+
+        newFut.requestMapping();
+
+        return newFut;
+    }
+
+    /**
+     * @param item Item.
+     * @param mappedClsName Mapped class name.
+     */
+    private IgniteCheckedException duplicateMappingException(MarshallerMappingItem item, String mappedClsName) {
+        return new IgniteCheckedException("Duplicate ID [platformId="
+                + item.platformId()
+                + ", typeId="
+                + item.typeId()
+                + ", oldCls="
+                + mappedClsName
+                + ", newCls="
+                + item.className() + "]");
+    }
+
+    /** */
+    public void markStopping() {
+        stopping = true;
+    }
+
+    /** */
+    public boolean stopping() {
+        return stopping;
+    }
+
+    /**
+     * Future to wait for mapping exchange result to arrive. Removes itself from map when completed.
+     */
+    private class MappingExchangeResultFuture extends GridFutureAdapter<MappingExchangeResult> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final MarshallerMappingItem mappingItem;
+
+        /**
+         * @param mappingItem Mapping item.
+         */
+        private MappingExchangeResultFuture(MarshallerMappingItem mappingItem) {
+            this.mappingItem = mappingItem;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable MappingExchangeResult res, @Nullable Throwable err) {
+            assert res != null;
+
+            boolean done = super.onDone(res, null);
+
+            if (done)
+                mappingExchSyncMap.remove(mappingItem, this);
+
+            return done;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingRequestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingRequestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingRequestMessage.java
new file mode 100644
index 0000000..ba2afce
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingRequestMessage.java
@@ -0,0 +1,146 @@
+/*
+ * 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.processors.marshaller;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ * Client node receives discovery messages in asynchronous mode
+ * so it is possible that all server nodes already accepted new mapping but clients are unaware about it.
+ *
+ * In this case it is possible for client node to receive a request to perform some operation on such class
+ * client doesn't know about its mapping.
+ * Upon receiving such request client sends an explicit {@link MissingMappingRequestMessage} mapping request
+ * to one of server nodes using CommunicationSPI and waits for {@link MissingMappingResponseMessage} response.
+ *
+ * If server node where mapping request was sent to leaves the cluster for some reason
+ * mapping request gets automatically resent to the next alive server node in topology.
+ */
+public class MissingMappingRequestMessage implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private byte platformId;
+
+    /** */
+    private int typeId;
+
+    /**
+     * Default constructor.
+     */
+    public MissingMappingRequestMessage() {
+        //No-op.
+    }
+
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     */
+    MissingMappingRequestMessage(byte platformId, int typeId) {
+        this.platformId = platformId;
+        this.typeId = typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByte("platformId", platformId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeInt("typeId", typeId))
+                    return false;
+
+                writer.incrementState();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                platformId = reader.readByte("platformId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                typeId = reader.readInt("typeId");
+
+                if (!reader.isLastRead())
+                    return false;
+        }
+
+        return reader.afterMessageRead(MissingMappingRequestMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 120;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** */
+    public byte platformId() {
+        return platformId;
+    }
+
+    /** */
+    public int typeId() {
+        return typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MissingMappingRequestMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingResponseMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingResponseMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingResponseMessage.java
new file mode 100644
index 0000000..d468302
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MissingMappingResponseMessage.java
@@ -0,0 +1,169 @@
+/*
+ * 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.processors.marshaller;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ * On receiving a {@link MissingMappingRequestMessage} mapping request server node looks up class name
+ * for requested platformId and typeId in its local marshaller cache and sends back
+ * a {@link MissingMappingResponseMessage} mapping response with resolved class name.
+ */
+public class MissingMappingResponseMessage implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private byte platformId;
+
+    /** */
+    private int typeId;
+
+    /** */
+    private String clsName;
+
+    /**
+     * Default constructor.
+     */
+    public MissingMappingResponseMessage() {
+    }
+
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     * @param clsName Class name.
+     */
+    MissingMappingResponseMessage(byte platformId, int typeId, String clsName) {
+        this.platformId = platformId;
+        this.typeId = typeId;
+        this.clsName = clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByte("platformId", platformId))
+                    return false;
+
+                writer.incrementState();
+            case 1:
+                if (!writer.writeInt("typeId", typeId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeString("clsName", clsName))
+                    return false;
+
+                writer.incrementState();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                platformId = reader.readByte("platformId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                typeId = reader.readInt("typeId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                clsName = reader.readString("clsName");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+        }
+
+        return reader.afterMessageRead(MissingMappingResponseMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 121;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /**
+     *
+     */
+    public byte platformId() {
+        return platformId;
+    }
+
+    /**
+     *
+     */
+    public int typeId() {
+        return typeId;
+    }
+
+    /**
+     *
+     */
+    public String className() {
+        return clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MissingMappingResponseMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
index 526e222..1a45f7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
@@ -17,6 +17,22 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
+import java.lang.reflect.Field;
+import java.math.BigDecimal;
+import java.security.Timestamp;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import javax.cache.CacheException;
+import javax.cache.event.CacheEntryEvent;
+import javax.cache.event.CacheEntryListenerException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -42,8 +58,8 @@ import org.apache.ignite.internal.processors.platform.memory.PlatformInputStream
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemoryUtils;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
-import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -51,23 +67,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.logger.NullLogger;
 import org.jetbrains.annotations.Nullable;
 
-import javax.cache.CacheException;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import java.math.BigDecimal;
-import java.security.Timestamp;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PREFIX;
 
 /**
@@ -844,25 +843,16 @@ public class PlatformUtils {
      */
     @SuppressWarnings("deprecation")
     public static GridBinaryMarshaller marshaller() {
-        try {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            BinaryContext ctx =
-                new BinaryContext(BinaryNoopMetadataHandler.instance(), cfg, new NullLogger());
-
-            BinaryMarshaller marsh = new BinaryMarshaller();
+        BinaryContext ctx =
+            new BinaryContext(BinaryNoopMetadataHandler.instance(), new IgniteConfiguration(), new NullLogger());
 
-            String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome());
+        BinaryMarshaller marsh = new BinaryMarshaller();
 
-            marsh.setContext(new MarshallerContextImpl(workDir, null));
+        marsh.setContext(new MarshallerContextImpl(null));
 
-            ctx.configure(marsh, new IgniteConfiguration());
+        ctx.configure(marsh, new IgniteConfiguration());
 
-            return new GridBinaryMarshaller(ctx);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
-        }
+        return new GridBinaryMarshaller(ctx);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
index 560d474..ffc1ded 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
@@ -39,8 +39,13 @@ import org.apache.ignite.plugin.Extension;
 import org.apache.ignite.plugin.ExtensionRegistry;
 import org.apache.ignite.plugin.PluginContext;
 import org.apache.ignite.plugin.PluginProvider;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.PLUGIN;
+
 /**
  *
  */
@@ -148,44 +153,87 @@ public class IgnitePluginProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
-        return DiscoveryDataExchangeType.PLUGIN;
+        return PLUGIN;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        Serializable pluginsData = getDiscoveryData(dataBag.joiningNodeId());
+
+        if (pluginsData != null)
+            dataBag.addJoiningNodeData(PLUGIN.ordinal(), pluginsData);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Serializable collectDiscoveryData(UUID nodeId) {
-        HashMap<String, Serializable> discData = null;
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        Serializable pluginsData = getDiscoveryData(dataBag.joiningNodeId());
+
+        if (pluginsData != null)
+            dataBag.addNodeSpecificData(PLUGIN.ordinal(), pluginsData);
+    }
+
+    /**
+     * @param joiningNodeId Joining node id.
+     */
+    private Serializable getDiscoveryData(UUID joiningNodeId) {
+        HashMap<String, Serializable> pluginsData = null;
 
         for (Map.Entry<String, PluginProvider> e : plugins.entrySet()) {
-            Serializable data = e.getValue().provideDiscoveryData(nodeId);
+            Serializable data = e.getValue().provideDiscoveryData(joiningNodeId);
 
             if (data != null) {
-                if (discData == null)
-                    discData = new HashMap<>();
+                if (pluginsData == null)
+                    pluginsData = new HashMap<>();
 
-                discData.put(e.getKey(), data);
+                pluginsData.put(e.getKey(), data);
             }
         }
 
-        return discData;
+        return pluginsData;
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID nodeId, UUID rmtNodeId, Serializable data) {
-        Map<String, Serializable> discData = (Map<String, Serializable>)data;
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
+        if (data.hasJoiningNodeData()) {
+            Map<String, Serializable> pluginsData = (Map<String, Serializable>) data.joiningNodeData();
 
-        if (discData != null) {
-            for (Map.Entry<String, Serializable> e : discData.entrySet()) {
-                PluginProvider provider = plugins.get(e.getKey());
+            applyPluginsData(data.joiningNodeId(), pluginsData);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        Map<UUID, Serializable> nodeSpecificData = data.nodeSpecificData();
+
+        if (nodeSpecificData != null) {
+            UUID joiningNodeId = data.joiningNodeId();
+
+            for (Serializable v : nodeSpecificData.values()) {
+                if (v != null) {
+                    Map<String, Serializable> pluginsData = (Map<String, Serializable>) v;
 
-                if (provider != null)
-                    provider.receiveDiscoveryData(nodeId, e.getValue());
-                else
-                    U.warn(log, "Received discovery data for unknown plugin: " + e.getKey());
+                    applyPluginsData(joiningNodeId, pluginsData);
+                }
             }
         }
     }
 
     /**
+     * @param nodeId Node id.
+     * @param pluginsData Plugins data.
+     */
+    private void applyPluginsData(UUID nodeId, Map<String, Serializable> pluginsData) {
+        for (Map.Entry<String, Serializable> e : pluginsData.entrySet()) {
+            PluginProvider provider = plugins.get(e.getKey());
+
+            if (provider != null)
+                provider.receiveDiscoveryData(nodeId, e.getValue());
+            else
+                U.warn(log, "Received discovery data for unknown plugin: " + e.getKey());
+        }
+    }
+
+    /**
      * Print plugins information.
      */
     private void ackPluginsInfo() {
@@ -227,7 +275,7 @@ public class IgnitePluginProcessor extends GridProcessorAdapter {
         /**
          * @return Map extension interface to array of implementation.
          */
-        public Map<Class<?>, Object[]> createExtensionMap() {
+        Map<Class<?>, Object[]> createExtensionMap() {
             Map<Class<?>, Object[]> extensions = new HashMap<>(extensionsCollector.size() * 2, 0.5f);
 
             for (Map.Entry<Class<?>, List<Object>> entry : extensionsCollector.entrySet()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
index 89140b9..f42815b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/pool/PoolProcessor.java
@@ -118,11 +118,6 @@ public class PoolProcessor extends GridProcessorAdapter {
 
                 return ctx.utilityCachePool();
 
-            case GridIoPolicy.MARSH_CACHE_POOL:
-                assert ctx.marshallerCachePool() != null : "Marshaller cache pool is not configured.";
-
-                return ctx.marshallerCachePool();
-
             case GridIoPolicy.IGFS_POOL:
                 assert ctx.getIgfsExecutorService() != null : "IGFS pool is not configured.";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index d26242d..986fff7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -69,6 +69,8 @@ import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.query.CacheQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
+import org.apache.ignite.internal.processors.marshaller.MappingAcceptedMessage;
+import org.apache.ignite.internal.processors.marshaller.MappingProposedMessage;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridEmptyIterator;
@@ -1580,12 +1582,15 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 if (evt instanceof DiscoveryCustomEvent) {
                     DiscoveryCustomMessage msg = ((DiscoveryCustomEvent)evt).customMessage();
 
-                    topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion();
-
                     if (msg instanceof CacheAffinityChangeMessage) {
                         if (!((CacheAffinityChangeMessage)msg).exchangeNeeded())
                             return;
                     }
+
+                    if (msg instanceof MappingProposedMessage || msg instanceof MappingAcceptedMessage)
+                        return;
+
+                    topVer = ((DiscoveryCustomEvent)evt).affinityTopologyVersion();
                 }
                 else
                     topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java
index 252ddd6..6898c36 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/MarshallerContext.java
@@ -26,23 +26,36 @@ public interface MarshallerContext {
     /**
      * Registers class with provided type ID.
      *
-     * @param id Type ID.
-     * @param cls Class.
+     * @param typeId Type ID.
+     * @param platformId id of a platform to register class.
+     * @param clsName class name.
      * @return Whether class was registered.
      * @throws IgniteCheckedException In case of error.
      */
-    public boolean registerClass(int id, Class cls) throws IgniteCheckedException;
+    public boolean registerClassName(byte platformId, int typeId, String clsName) throws IgniteCheckedException;
 
     /**
      * Gets class for provided type ID.
      *
-     * @param id Type ID.
+     * @param typeId Type ID.
      * @param ldr Class loader.
      * @return Class.
      * @throws ClassNotFoundException If class was not found.
      * @throws IgniteCheckedException In case of any other error.
      */
-    public Class getClass(int id, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException;
+    public Class getClass(int typeId, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException;
+
+
+    /**
+     * Gets class name for provided (platformId, typeId) pair.
+     *
+     * @param platformId id of a platform the class was registered for.
+     * @param typeId Type ID.
+     * @return Class name
+     * @throws ClassNotFoundException If class was not found.
+     * @throws IgniteCheckedException In case of any other error.
+     */
+    public String getClassName(byte platformId, int typeId) throws ClassNotFoundException, IgniteCheckedException;
 
     /**
      * Checks whether the given type is a system one - JDK class or Ignite class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index 923f385..60a627a 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -35,6 +35,8 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
+
 /**
  * Miscellaneous utility methods to facilitate {@link OptimizedMarshaller}.
  */
@@ -159,7 +161,7 @@ class OptimizedMarshallerUtils {
         try {
             mapOff = GridUnsafe.objectFieldOffset(HashSet.class.getDeclaredField("map"));
         }
-        catch (NoSuchFieldException e) {
+        catch (NoSuchFieldException ignored) {
             try {
                 // Workaround for legacy IBM JRE.
                 mapOff = GridUnsafe.objectFieldOffset(HashSet.class.getDeclaredField("backingMap"));
@@ -203,7 +205,7 @@ class OptimizedMarshallerUtils {
             boolean registered;
 
             try {
-                registered = ctx.registerClass(typeId, cls);
+                registered = ctx.registerClassName(JAVA_ID, typeId, cls.getName());
             }
             catch (IgniteCheckedException e) {
                 throw new IOException("Failed to register class: " + cls.getName(), e);
@@ -246,7 +248,7 @@ class OptimizedMarshallerUtils {
      * Gets descriptor for provided ID.
      *
      * @param clsMap Class descriptors by class map.
-     * @param id ID.
+     * @param typeId ID.
      * @param ldr Class loader.
      * @param ctx Context.
      * @param mapper ID mapper.
@@ -256,17 +258,17 @@ class OptimizedMarshallerUtils {
      */
     static OptimizedClassDescriptor classDescriptor(
         ConcurrentMap<Class, OptimizedClassDescriptor> clsMap,
-        int id,
+        int typeId,
         ClassLoader ldr,
         MarshallerContext ctx,
         OptimizedMarshallerIdMapper mapper) throws IOException, ClassNotFoundException {
         Class cls;
 
         try {
-            cls = ctx.getClass(id, ldr);
+            cls = ctx.getClass(typeId, ldr);
         }
         catch (IgniteCheckedException e) {
-            throw new IOException("Failed to resolve class for ID: " + id, e);
+            throw new IOException("Failed to resolve class for ID: " + typeId, e);
         }
 
         OptimizedClassDescriptor desc = clsMap.get(cls);
@@ -307,7 +309,7 @@ class OptimizedMarshallerUtils {
                     }
                 }
             }
-            catch (NoSuchFieldException e) {
+            catch (NoSuchFieldException ignored) {
                 // No-op.
             }
             catch (IllegalAccessException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
old mode 100644
new mode 100755

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
new file mode 100644
index 0000000..96df255
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoveryDataBag.java
@@ -0,0 +1,299 @@
+/*
+ * 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.spi.discovery;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.internal.GridComponent;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Provides interface for {@link GridComponent} to collect and exchange discovery data both on
+ * joining node and on cluster nodes.
+ *
+ * It only organizes interaction with components and doesn't provide any capabilities of converting collected data
+ * into formats eligible for transmitting over media (like marshalling, compressing and so on).
+ */
+public class DiscoveryDataBag {
+    /**
+     * Facade interface representing {@link DiscoveryDataBag} object with discovery data from joining node.
+     */
+    public interface JoiningNodeDiscoveryData {
+        /** */
+        UUID joiningNodeId();
+
+        /** */
+        boolean hasJoiningNodeData();
+
+        /** */
+        Serializable joiningNodeData();
+    }
+
+    /**
+     * Facade interface representing {@link DiscoveryDataBag} object with discovery data collected in the grid.
+     */
+    public interface GridDiscoveryData {
+        /** */
+        UUID joiningNodeId();
+
+        /** */
+        Serializable commonData();
+
+        /** */
+        Map<UUID, Serializable> nodeSpecificData();
+    }
+
+    /**
+     *
+     */
+    private final class JoiningNodeDiscoveryDataImpl implements JoiningNodeDiscoveryData {
+        /** */
+        private int cmpId;
+
+        /** {@inheritDoc} */
+        @Override public UUID joiningNodeId() {
+            return joiningNodeId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasJoiningNodeData() {
+            return joiningNodeData.containsKey(cmpId);
+        }
+
+        /** {@inheritDoc} */
+        @Override @Nullable public Serializable joiningNodeData() {
+            return joiningNodeData.get(cmpId);
+        }
+
+        /**
+         * @param cmpId Cmp id.
+         */
+        private void setComponentId(int cmpId) {
+            this.cmpId = cmpId;
+        }
+    }
+
+    /**
+     *
+     */
+    private final class GridDiscoveryDataImpl implements GridDiscoveryData {
+        /** */
+        private int cmpId;
+
+        /** */
+        private Map<UUID, Serializable> nodeSpecificData
+                = new LinkedHashMap<>(DiscoveryDataBag.this.nodeSpecificData.size());
+
+        /** {@inheritDoc} */
+        @Override public UUID joiningNodeId() {
+            return joiningNodeId;
+        }
+
+        /** {@inheritDoc} */
+        @Override @Nullable public Serializable commonData() {
+            if (commonData != null)
+                return commonData.get(cmpId);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<UUID, Serializable> nodeSpecificData() {
+            return nodeSpecificData;
+        }
+
+        /**
+         * @param cmpId component ID.
+         */
+        private void componentId(int cmpId) {
+            this.cmpId = cmpId;
+
+            reinitNodeSpecData(cmpId);
+        }
+
+        /**
+         * @param cmpId component ID.
+         */
+        private void reinitNodeSpecData(int cmpId) {
+            nodeSpecificData.clear();
+
+            for (Map.Entry<UUID, Map<Integer, Serializable>> e : DiscoveryDataBag.this.nodeSpecificData.entrySet()) {
+                if (e.getValue() != null && e.getValue().containsKey(cmpId))
+                    nodeSpecificData.put(e.getKey(), e.getValue().get(cmpId));
+            }
+        }
+    }
+
+    /** Used for collecting node-specific data from component.
+     * As component may not know about nodeId it is running on, when component adds node-specific data,
+     * it is firstly collected under this key and then moved to another map with a correct UUID key.
+     */
+    private static final UUID DEFAULT_KEY = null;
+
+    /** */
+    private UUID joiningNodeId;
+
+    /**
+     * Component IDs with already initialized common discovery data.
+     */
+    private Set<Integer> cmnDataInitializedCmps;
+
+    /** */
+    private Map<Integer, Serializable> joiningNodeData = new HashMap<>();
+
+    /** */
+    private Map<Integer, Serializable> commonData = new HashMap<>();
+
+    /** */
+    private Map<UUID, Map<Integer, Serializable>> nodeSpecificData = new LinkedHashMap<>();
+
+    /** */
+    private JoiningNodeDiscoveryDataImpl newJoinerData;
+
+    /** */
+    private GridDiscoveryDataImpl gridData;
+
+    /**
+     * @param joiningNodeId Joining node id.
+     */
+    public DiscoveryDataBag(UUID joiningNodeId) {
+        this.joiningNodeId = joiningNodeId;
+    }
+
+    /**
+     * @param joiningNodeId Joining node id.
+     * @param cmnDataInitializedCmps Component IDs with already initialized common discovery data.
+     */
+    public DiscoveryDataBag(UUID joiningNodeId, Set<Integer> cmnDataInitializedCmps) {
+        this.joiningNodeId = joiningNodeId;
+        this.cmnDataInitializedCmps = cmnDataInitializedCmps;
+    }
+
+    /**
+     *
+     */
+    public UUID joiningNodeId() {
+        return joiningNodeId;
+    }
+
+    /**
+     * @param cmpId component ID.
+     */
+    public GridDiscoveryData gridDiscoveryData(int cmpId) {
+        if (gridData == null)
+            gridData = new GridDiscoveryDataImpl();
+
+        gridData.componentId(cmpId);
+
+        return gridData;
+    }
+
+    /**
+     * @param cmpId component ID.
+     */
+    public JoiningNodeDiscoveryData newJoinerDiscoveryData(int cmpId) {
+        if (newJoinerData == null)
+            newJoinerData = new JoiningNodeDiscoveryDataImpl();
+
+        newJoinerData.setComponentId(cmpId);
+
+        return newJoinerData;
+    }
+
+    /**
+     * @param cmpId component ID.
+     * @param data Data.
+     */
+    public void addJoiningNodeData(Integer cmpId, Serializable data) {
+        joiningNodeData.put(cmpId, data);
+    }
+
+    /**
+     * @param cmpId component ID.
+     * @param data Data.
+     */
+    public void addGridCommonData(Integer cmpId, Serializable data) {
+        commonData.put(cmpId, data);
+    }
+
+    /**
+     * @param cmpId component ID.
+     * @param data Data.
+     */
+    public void addNodeSpecificData(Integer cmpId, Serializable data) {
+        if (!nodeSpecificData.containsKey(DEFAULT_KEY))
+            nodeSpecificData.put(DEFAULT_KEY, new HashMap<Integer, Serializable>());
+
+        nodeSpecificData.get(DEFAULT_KEY).put(cmpId, data);
+    }
+
+    /**
+     * @param cmpId component ID.
+     */
+    public boolean commonDataCollectedFor(Integer cmpId) {
+        assert cmnDataInitializedCmps != null;
+
+        return cmnDataInitializedCmps.contains(cmpId);
+    }
+
+    /**
+     * @param joinNodeData Joining node data.
+     */
+    public void joiningNodeData(Map<Integer, Serializable> joinNodeData) {
+        joiningNodeData.putAll(joinNodeData);
+    }
+
+    /**
+     * @param cmnData Cmn data.
+     */
+    public void commonData(Map<Integer, Serializable> cmnData) {
+        commonData.putAll(cmnData);
+    }
+
+    /**
+     * @param nodeSpecData Node specific data.
+     */
+    public void nodeSpecificData(Map<UUID, Map<Integer, Serializable>> nodeSpecData) {
+        nodeSpecificData.putAll(nodeSpecData);
+    }
+
+    /**
+     *
+     */
+    public Map<Integer, Serializable> joiningNodeData() {
+        return joiningNodeData;
+    }
+
+    /**
+     *
+     */
+    public Map<Integer, Serializable> commonData() {
+        return commonData;
+    }
+
+    /**
+     *
+     */
+    @Nullable public Map<Integer, Serializable> localNodeSpecificData() {
+        return nodeSpecificData.get(DEFAULT_KEY);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
index e591470..d56c943 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.spi.discovery;
 
-import java.io.Serializable;
-import java.util.Map;
-import java.util.UUID;
-
 /**
  * Handler for initial data exchange between Ignite nodes. Data exchange
  * is initiated by a new node when it tries to join topology and finishes
@@ -32,17 +28,14 @@ public interface DiscoverySpiDataExchange {
      * on new node that joins topology to transfer its data to existing
      * nodes and on all existing nodes to transfer their data to new node.
      *
-     * @param joiningNodeId ID of new node that joins topology.
-     * @return Collection of discovery data objects from different components.
+     * @param dataBag {@link DiscoveryDataBag} object managing discovery data during node joining process.
      */
-    public Map<Integer, Serializable> collect(UUID joiningNodeId);
+    public DiscoveryDataBag collect(DiscoveryDataBag dataBag);
 
     /**
      * Notifies discovery manager about data received from remote node.
      *
-     * @param joiningNodeId ID of new node that joins topology.
-     * @param nodeId ID of the node provided data.
-     * @param data Collection of discovery data objects from different components.
+     * @param dataBag Collection of discovery data objects from different components.
      */
-    public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data);
+    public void onExchange(DiscoveryDataBag dataBag);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 8928f28..a5cedf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -70,6 +70,7 @@ import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper;
 import org.apache.ignite.spi.IgniteSpiThread;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
@@ -623,7 +624,9 @@ class ClientImpl extends TcpDiscoveryImpl {
                     if (locNode.order() > 0)
                         node = locNode.clientReconnectNode();
 
-                    msg = new TcpDiscoveryJoinRequestMessage(node, spi.collectExchangeData(getLocalNodeId()));
+                    msg = new TcpDiscoveryJoinRequestMessage(
+                            node,
+                            spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId())));
                 }
                 else
                     msg = new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId, lastMsgId);
@@ -1048,7 +1051,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
-            TcpDiscoveryAbstractMessage msg;
+            TcpDiscoveryAbstractMessage msg = null;
 
             while (!Thread.currentThread().isInterrupted()) {
                 Socket sock;
@@ -1062,7 +1065,8 @@ class ClientImpl extends TcpDiscoveryImpl {
                         continue;
                     }
 
-                    msg = queue.poll();
+                    if (msg == null)
+                        msg = queue.poll();
 
                     if (msg == null) {
                         mux.wait();
@@ -1661,11 +1665,10 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (log.isDebugEnabled())
                             log.debug("Added new node to topology: " + node);
 
-                        Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
+                        DiscoveryDataPacket dataPacket = msg.gridDiscoveryData();
 
-                        if (data != null)
-                            spi.onExchange(newNodeId, newNodeId, data,
-                                U.resolveClassLoader(spi.ignite().configuration()));
+                        if (dataPacket != null && dataPacket.hasJoiningNodeData())
+                            spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
                     }
                 }
                 else {
@@ -1684,13 +1687,10 @@ class ClientImpl extends TcpDiscoveryImpl {
 
             if (getLocalNodeId().equals(msg.nodeId())) {
                 if (joining()) {
-                    Map<UUID, Map<Integer, byte[]>> dataMap = msg.clientDiscoData();
+                    DiscoveryDataPacket dataContainer = msg.clientDiscoData();
 
-                    if (dataMap != null) {
-                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                            spi.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(),
-                                U.resolveClassLoader(spi.ignite().configuration()));
-                    }
+                    if (dataContainer != null)
+                        spi.onExchange(dataContainer, U.resolveClassLoader(spi.ignite().configuration()));
 
                     locNode.setAttributes(msg.clientNodeAttributes());
                     locNode.visible(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 7cc35e4..df782e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -100,6 +100,7 @@ import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper;
 import org.apache.ignite.spi.IgniteSpiThread;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNodesRing;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState;
@@ -964,7 +965,7 @@ class ServerImpl extends TcpDiscoveryImpl {
     @SuppressWarnings({"BusyWait"})
     private boolean sendJoinRequestMessage() throws IgniteSpiException {
         TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
-            spi.collectExchangeData(getLocalNodeId()));
+            spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId())));
 
         // Time when it has been detected, that addresses from IP finder do not respond.
         long noResStart = 0;
@@ -1933,7 +1934,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 // Do not need this data for client reconnect.
-                addedMsg.oldNodesDiscoveryData(null);
+                if (addedMsg.gridDiscoveryData() != null)
+                    addedMsg.clearDiscoveryData();
             }
             else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) {
                 TcpDiscoveryNodeAddFinishedMessage addFinishMsg = (TcpDiscoveryNodeAddFinishedMessage)msg;
@@ -1943,44 +1945,25 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     msg = addFinishMsg;
 
-                    Map<UUID, Map<Integer, byte[]>> discoData = addFinishMsg.clientDiscoData();
+                    DiscoveryDataPacket discoData = addFinishMsg.clientDiscoData();
 
-                    Set<UUID> replaced = null;
+                    Set<Integer> mrgdCmnData = new HashSet<>();
+                    Set<UUID> mrgdSpecData = new HashSet<>();
+
+                    boolean allMerged = false;
 
                     for (TcpDiscoveryAbstractMessage msg0 : msgs) {
+
                         if (msg0 instanceof TcpDiscoveryNodeAddFinishedMessage) {
-                            Map<UUID, Map<Integer, byte[]>> existingDiscoData =
+                            DiscoveryDataPacket existingDiscoData =
                                 ((TcpDiscoveryNodeAddFinishedMessage)msg0).clientDiscoData();
 
-                            // Check if already stored message contains the same data to do not store copies multiple times.
-                            if (existingDiscoData != null) {
-                                for (Map.Entry<UUID, Map<Integer, byte[]>> e : discoData.entrySet()) {
-                                    UUID nodeId = e.getKey();
-
-                                    if (F.contains(replaced, nodeId))
-                                        continue;
-
-                                    Map<Integer, byte[]> existingData = existingDiscoData.get(e.getKey());
-
-                                    if (existingData != null && mapsEqual(e.getValue(), existingData)) {
-                                        e.setValue(existingData);
-
-                                        if (replaced == null)
-                                            replaced = new HashSet<>();
-
-                                        boolean add = replaced.add(nodeId);
-
-                                        assert add;
-
-                                        if (replaced.size() == discoData.size())
-                                            break;
-                                    }
-                                }
-
-                                if (replaced != null && replaced.size() == discoData.size())
-                                    break;
-                            }
+                            if (existingDiscoData != null)
+                                allMerged = discoData.mergeDataFrom(existingDiscoData, mrgdCmnData, mrgdSpecData);
                         }
+
+                        if (allMerged)
+                            break;
                     }
                 }
             }
@@ -2011,29 +1994,6 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
-         * @param m1 Map 1.
-         * @param m2 Map 2.
-         * @return {@code True} if maps contain the same data.
-         */
-        private boolean mapsEqual(Map<Integer, byte[]> m1, Map<Integer, byte[]> m2) {
-            if (m1 == m2)
-                return true;
-
-            if (m1.size() == m2.size()) {
-                for (Map.Entry<Integer, byte[]> e : m1.entrySet()) {
-                    byte[] data = m2.get(e.getKey());
-
-                    if (!Arrays.equals(e.getValue(), data))
-                        return false;
-                }
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
          * Gets messages starting from provided ID (exclusive). If such
          * message is not found, {@code null} is returned (this indicates
          * a failure condition when it was already removed from queue).
@@ -3748,8 +3708,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (log.isDebugEnabled())
                     log.debug("Internal order has been assigned to node: " + node);
 
+                DiscoveryDataPacket data = msg.gridDiscoveryData();
+
                 TcpDiscoveryNodeAddedMessage nodeAddedMsg = new TcpDiscoveryNodeAddedMessage(locNodeId,
-                    node, msg.discoveryData(), spi.gridStartTime);
+                    node, data, spi.gridStartTime);
 
                 nodeAddedMsg.client(msg.client());
 
@@ -3970,7 +3932,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         node.id());
 
                     if (node.isClient()) {
-                        addFinishMsg.clientDiscoData(msg.oldNodesDiscoveryData());
+                        addFinishMsg.clientDiscoData(msg.gridDiscoveryData());
 
                         addFinishMsg.clientNodeAttributes(node.attributes());
                     }
@@ -4100,12 +4062,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (topChanged) {
                     assert !node.visible() : "Added visible node [node=" + node + ", locNode=" + locNode + ']';
 
-                    Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
+                    DiscoveryDataPacket dataPacket = msg.gridDiscoveryData();
 
-                    if (data != null)
-                        spi.onExchange(node.id(), node.id(), data, U.resolveClassLoader(spi.ignite().configuration()));
+                    if (dataPacket.hasJoiningNodeData())
+                        spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
 
-                    msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
+                    spi.collectExchangeData(dataPacket);
 
                     processMessageFailedNodes(msg);
                 }
@@ -4116,8 +4078,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             if (msg.verified() && locNodeId.equals(node.id())) {
-                // Discovery data.
-                Map<UUID, Map<Integer, byte[]>> dataMap;
+                DiscoveryDataPacket dataPacket;
 
                 synchronized (mux) {
                     if (spiState == CONNECTING && locNode.internalOrder() != node.internalOrder()) {
@@ -4194,7 +4155,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             if (log.isDebugEnabled())
                                 log.debug("Restored topology from node added message: " + ring);
 
-                            dataMap = msg.oldNodesDiscoveryData();
+                            dataPacket = msg.gridDiscoveryData();
 
                             topHist.clear();
                             topHist.putAll(msg.topologyHistory());
@@ -4227,11 +4188,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 // Notify outside of synchronized block.
-                if (dataMap != null) {
-                    for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                        spi.onExchange(node.id(), entry.getKey(), entry.getValue(),
-                            U.resolveClassLoader(spi.ignite().configuration()));
-                }
+                if (dataPacket != null)
+                    spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
 
                 processMessageFailedNodes(msg);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index db85cc4..1f2f328 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -22,7 +22,6 @@ import java.io.EOFException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.Serializable;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.Socket;
@@ -51,7 +50,6 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.GridComponent;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -77,6 +75,7 @@ import org.apache.ignite.spi.IgniteSpiOperationTimeoutException;
 import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper;
 import org.apache.ignite.spi.IgniteSpiTimeoutObject;
 import org.apache.ignite.spi.IgniteSpiVersionCheckException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
@@ -85,6 +84,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiHistorySupport;
 import org.apache.ignite.spi.discovery.DiscoverySpiListener;
 import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
 import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryStatistics;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -1679,66 +1679,49 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
-     * @param nodeId Node ID.
-     * @return Marshalled exchange data.
+     * @param dataPacket Data packet.
      */
-    protected Map<Integer, byte[]> collectExchangeData(UUID nodeId) {
+    DiscoveryDataPacket collectExchangeData(DiscoveryDataPacket dataPacket) {
         if (locNode.isDaemon())
-            return Collections.emptyMap();
+            return dataPacket;
 
-        Map<Integer, Serializable> data = exchange.collect(nodeId);
+        assert dataPacket != null;
+        assert dataPacket.joiningNodeId() != null;
 
-        assert data != null;
-
-        Map<Integer, byte[]> data0 = U.newHashMap(data.size());
+        //create data bag, pass it to exchange.collect
+        DiscoveryDataBag dataBag = dataPacket.bagForDataCollection();
 
-        for (Map.Entry<Integer, Serializable> entry : data.entrySet()) {
-            try {
-                byte[] bytes = U.marshal(marshaller(), entry.getValue());
+        exchange.collect(dataBag);
 
-                data0.put(entry.getKey(), bytes);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to marshal discovery data " +
-                    "[comp=" + entry.getKey() + ", data=" + entry.getValue() + ']', e);
-            }
-        }
+        //marshall collected bag into packet, return packet
+        if (dataPacket.joiningNodeId().equals(locNode.id()))
+            dataPacket.marshalJoiningNodeData(dataBag, marshaller(), log);
+        else
+            dataPacket.marshalGridNodeData(dataBag, locNode.id(), marshaller(), log);
 
-        return data0;
+        return dataPacket;
     }
 
     /**
-     * @param joiningNodeID Joining node ID.
-     * @param nodeId Remote node ID for which data is provided.
-     * @param data Collection of marshalled discovery data objects from different components.
+     * @param dataPacket object holding discovery data collected during discovery process.
      * @param clsLdr Class loader.
      */
-    protected void onExchange(UUID joiningNodeID,
-        UUID nodeId,
-        Map<Integer, byte[]> data,
-        ClassLoader clsLdr)
-    {
+    protected void onExchange(DiscoveryDataPacket dataPacket, ClassLoader clsLdr) {
         if (locNode.isDaemon())
             return;
 
-        Map<Integer, Serializable> data0 = U.newHashMap(data.size());
+        assert dataPacket != null;
+        assert dataPacket.joiningNodeId() != null;
 
-        for (Map.Entry<Integer, byte[]> entry : data.entrySet()) {
-            try {
-                Serializable compData = U.unmarshal(marshaller(), entry.getValue(), clsLdr);
+        DiscoveryDataBag dataBag;
+
+        if (dataPacket.joiningNodeId().equals(locNode.id()))
+            dataBag = dataPacket.unmarshalGridData(marshaller(), clsLdr, locNode.isClient(), log);
+        else
+            dataBag = dataPacket.unmarshalJoiningNodeData(marshaller(), clsLdr, locNode.isClient(), log);
 
-                data0.put(entry.getKey(), compData);
-            }
-            catch (IgniteCheckedException e) {
-                if (GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC.ordinal() == entry.getKey() &&
-                    X.hasCause(e, ClassNotFoundException.class) && locNode.isClient())
-                    U.warn(log, "Failed to unmarshal continuous query remote filter on client node. Can be ignored.");
-                else
-                    U.error(log, "Failed to unmarshal discovery data for component: "  + entry.getKey(), e);
-            }
-        }
 
-        exchange.onExchange(joiningNodeID, nodeId, data0);
+        exchange.onExchange(dataBag);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/DiscoveryDataPacket.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/DiscoveryDataPacket.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/DiscoveryDataPacket.java
new file mode 100644
index 0000000..1134de6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/DiscoveryDataPacket.java
@@ -0,0 +1,345 @@
+/*
+ * 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.spi.discovery.tcp.internal;
+
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.Marshaller;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC;
+
+/**
+ * Carries discovery data in marshalled form
+ * and allows convenient way of converting it to and from {@link DiscoveryDataBag} objects.
+ */
+public class DiscoveryDataPacket implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final UUID joiningNodeId;
+
+    /** */
+    private Map<Integer, byte[]> joiningNodeData = new HashMap<>();
+
+    /** */
+    private Map<Integer, byte[]> commonData = new HashMap<>();
+
+    /** */
+    private Map<UUID, Map<Integer, byte[]>> nodeSpecificData = new LinkedHashMap<>();
+
+    /**
+     * @param joiningNodeId Joining node id.
+     */
+    public DiscoveryDataPacket(UUID joiningNodeId) {
+        this.joiningNodeId = joiningNodeId;
+    }
+
+    /**
+     *
+     */
+    public UUID joiningNodeId() {
+        return joiningNodeId;
+    }
+
+    /**
+     * @param bag Bag.
+     * @param nodeId Node id.
+     * @param marsh Marsh.
+     * @param log Logger.
+     */
+    public void marshalGridNodeData(DiscoveryDataBag bag, UUID nodeId, Marshaller marsh, IgniteLogger log) {
+        marshalData(bag.commonData(), commonData, marsh, log);
+
+        Map<Integer, Serializable> locNodeSpecificData = bag.localNodeSpecificData();
+
+        if (locNodeSpecificData != null) {
+            Map<Integer, byte[]> marshLocNodeSpecificData = U.newHashMap(locNodeSpecificData.size());
+
+            marshalData(locNodeSpecificData, marshLocNodeSpecificData, marsh, log);
+
+            filterDuplicatedData(marshLocNodeSpecificData);
+
+            if (!marshLocNodeSpecificData.isEmpty())
+                nodeSpecificData.put(nodeId, marshLocNodeSpecificData);
+        }
+    }
+
+    /**
+     * @param bag Bag.
+     * @param marsh Marsh.
+     * @param log Logger.
+     */
+    public void marshalJoiningNodeData(DiscoveryDataBag bag, Marshaller marsh, IgniteLogger log) {
+        marshalData(bag.joiningNodeData(), joiningNodeData, marsh, log);
+    }
+
+    /**
+     * @param marsh Marsh.
+     * @param clsLdr Class loader.
+     * @param clientNode Client node.
+     * @param log Logger.
+     */
+    public DiscoveryDataBag unmarshalGridData(
+            Marshaller marsh,
+            ClassLoader clsLdr,
+            boolean clientNode,
+            IgniteLogger log
+    ) {
+        DiscoveryDataBag dataBag = new DiscoveryDataBag(joiningNodeId);
+
+        if (commonData != null && !commonData.isEmpty()) {
+            Map<Integer, Serializable> unmarshCommonData = unmarshalData(commonData, marsh, clsLdr, clientNode, log);
+
+            dataBag.commonData(unmarshCommonData);
+        }
+
+        if (nodeSpecificData != null && !nodeSpecificData.isEmpty()) {
+            Map<UUID, Map<Integer, Serializable>> unmarshNodeSpecData = U.newLinkedHashMap(nodeSpecificData.size());
+
+            for (Map.Entry<UUID, Map<Integer, byte[]>> nodeBinEntry : nodeSpecificData.entrySet()) {
+                Map<Integer, byte[]> nodeBinData = nodeBinEntry.getValue();
+
+                if (nodeBinData == null || nodeBinData.isEmpty())
+                    continue;
+
+                Map<Integer, Serializable> unmarshData = unmarshalData(nodeBinData, marsh, clsLdr, clientNode, log);
+
+                unmarshNodeSpecData.put(nodeBinEntry.getKey(), unmarshData);
+            }
+
+            dataBag.nodeSpecificData(unmarshNodeSpecData);
+        }
+
+        return dataBag;
+    }
+
+    /**
+     * @param marsh Marsh.
+     * @param clsLdr Class loader.
+     * @param clientNode Client node.
+     * @param log Logger.
+     */
+    public DiscoveryDataBag unmarshalJoiningNodeData(
+            Marshaller marsh,
+            ClassLoader clsLdr,
+            boolean clientNode,
+            IgniteLogger log
+    ) {
+        DiscoveryDataBag dataBag = new DiscoveryDataBag(joiningNodeId);
+
+        if (joiningNodeData != null && !joiningNodeData.isEmpty()) {
+            Map<Integer, Serializable> unmarshJoiningNodeData = unmarshalData(
+                    joiningNodeData,
+                    marsh,
+                    clsLdr,
+                    clientNode,
+                    log);
+
+            dataBag.joiningNodeData(unmarshJoiningNodeData);
+        }
+
+        return dataBag;
+    }
+
+    /**
+     *
+     */
+    public boolean hasJoiningNodeData() {
+        return joiningNodeData != null && !joiningNodeData.isEmpty();
+    }
+
+    /**
+     * @param nodeId Node id.
+     */
+    public boolean hasDataFromNode(UUID nodeId) {
+        return nodeSpecificData.containsKey(nodeId);
+    }
+
+    /**
+     * @param existingDataPacket Existing data packet.
+     * @param mrgdCmnDataKeys Mrgd cmn data keys.
+     * @param mrgdSpecifDataKeys Mrgd specif data keys.
+     */
+    public boolean mergeDataFrom(
+            DiscoveryDataPacket existingDataPacket,
+            Collection<Integer> mrgdCmnDataKeys,
+            Collection<UUID> mrgdSpecifDataKeys
+    ) {
+        if (commonData.size() != mrgdCmnDataKeys.size()) {
+            for (Map.Entry<Integer, byte[]> e : commonData.entrySet()) {
+                if (!mrgdCmnDataKeys.contains(e.getKey())) {
+                    byte[] data = existingDataPacket.commonData.get(e.getKey());
+
+                    if (data != null && Arrays.equals(e.getValue(), data)) {
+                        e.setValue(data);
+
+                        boolean add = mrgdCmnDataKeys.add(e.getKey());
+
+                        assert add;
+
+                        if (mrgdCmnDataKeys.size() == commonData.size())
+                            break;
+                    }
+                }
+            }
+        }
+
+        if (nodeSpecificData.size() != mrgdSpecifDataKeys.size()) {
+            for (Map.Entry<UUID, Map<Integer, byte[]>> e : nodeSpecificData.entrySet()) {
+                if (!mrgdSpecifDataKeys.contains(e.getKey())) {
+                    Map<Integer, byte[]> data = existingDataPacket.nodeSpecificData.get(e.getKey());
+
+                    if (data != null && mapsEqual(e.getValue(), data)) {
+                        e.setValue(data);
+
+                        boolean add = mrgdSpecifDataKeys.add(e.getKey());
+
+                        assert add;
+
+                        if (mrgdSpecifDataKeys.size() == nodeSpecificData.size())
+                            break;
+                    }
+                }
+            }
+        }
+
+        return (mrgdCmnDataKeys.size() == commonData.size()) && (mrgdSpecifDataKeys.size() == nodeSpecificData.size());
+    }
+
+    /**
+     * @param m1 first map to compare.
+     * @param m2 second map to compare.
+     */
+    private boolean mapsEqual(Map<Integer, byte[]> m1, Map<Integer, byte[]> m2) {
+        if (m1 == m2)
+            return true;
+
+        if (m1.size() == m2.size()) {
+            for (Map.Entry<Integer, byte[]> e : m1.entrySet()) {
+                byte[] data = m2.get(e.getKey());
+
+                if (!Arrays.equals(e.getValue(), data))
+                    return false;
+            }
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * @param src Source.
+     * @param marsh Marsh.
+     * @param clsLdr Class loader.
+     * @param log Logger.
+     */
+    private Map<Integer, Serializable> unmarshalData(
+            Map<Integer, byte[]> src,
+            Marshaller marsh,
+            ClassLoader clsLdr,
+            boolean clientNode,
+            IgniteLogger log
+    ) {
+        Map<Integer, Serializable> res = U.newHashMap(src.size());
+
+        for (Map.Entry<Integer, byte[]> binEntry : src.entrySet()) {
+            try {
+                Serializable compData = marsh.unmarshal(binEntry.getValue(), clsLdr);
+                res.put(binEntry.getKey(), compData);
+            }
+            catch (IgniteCheckedException e) {
+                if (CONTINUOUS_PROC.ordinal() == binEntry.getKey() &&
+                        X.hasCause(e, ClassNotFoundException.class) && clientNode)
+                    U.warn(log, "Failed to unmarshal continuous query remote filter on client node. Can be ignored.");
+                else
+                    U.error(log, "Failed to unmarshal discovery data for component: "  + binEntry.getKey(), e);
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * @param src Source.
+     * @param target Target.
+     * @param marsh Marsh.
+     * @param log Logger.
+     */
+    private void marshalData(
+            Map<Integer, Serializable> src,
+            Map<Integer, byte[]> target,
+            Marshaller marsh,
+            IgniteLogger log
+    ) {
+        //may happen if nothing was collected from components,
+        // corresponding map (for common data or for node specific data) left null
+        if (src == null)
+            return;
+
+        for (Map.Entry<Integer, Serializable> entry : src.entrySet()) {
+            try {
+                target.put(entry.getKey(), marsh.marshal(entry.getValue()));
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to marshal discovery data " +
+                        "[comp=" + entry.getKey() + ", data=" + entry.getValue() + ']', e);
+            }
+        }
+    }
+
+    /**
+     * TODO https://issues.apache.org/jira/browse/IGNITE-4435
+     */
+    private void filterDuplicatedData(Map<Integer, byte[]> discoData) {
+        for (Map<Integer, byte[]> existingData : nodeSpecificData.values()) {
+            Iterator<Map.Entry<Integer, byte[]>> it = discoData.entrySet().iterator();
+
+            while (it.hasNext()) {
+                Map.Entry<Integer, byte[]> discoDataEntry = it.next();
+
+                byte[] curData = existingData.get(discoDataEntry.getKey());
+
+                if (Arrays.equals(curData, discoDataEntry.getValue()))
+                    it.remove();
+            }
+
+            if (discoData.isEmpty())
+                break;
+        }
+    }
+
+    /**
+     * Returns {@link DiscoveryDataBag} aware of components with already initialized common data
+     * (e.g. on nodes prior in cluster to the one where this method is called).
+     */
+    public DiscoveryDataBag bagForDataCollection() {
+        return new DiscoveryDataBag(joiningNodeId, commonData.keySet());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
index 22ffae8..24e3868 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryJoinRequestMessage.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
-import java.util.Map;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 
 /**
@@ -33,20 +33,20 @@ public class TcpDiscoveryJoinRequestMessage extends TcpDiscoveryAbstractMessage
     /** New node that wants to join the topology. */
     private final TcpDiscoveryNode node;
 
-    /** Discovery data. */
-    private final Map<Integer, byte[]> discoData;
+    /** Discovery data container. */
+    private final DiscoveryDataPacket dataPacket;
 
     /**
      * Constructor.
      *
      * @param node New node that wants to join.
-     * @param discoData Discovery data.
+     * @param dataPacket Discovery data.
      */
-    public TcpDiscoveryJoinRequestMessage(TcpDiscoveryNode node, Map<Integer, byte[]> discoData) {
+    public TcpDiscoveryJoinRequestMessage(TcpDiscoveryNode node, DiscoveryDataPacket dataPacket) {
         super(node.id());
 
         this.node = node;
-        this.discoData = discoData;
+        this.dataPacket = dataPacket;
     }
 
     /**
@@ -59,10 +59,10 @@ public class TcpDiscoveryJoinRequestMessage extends TcpDiscoveryAbstractMessage
     }
 
     /**
-     * @return Discovery data.
+     *
      */
-    public Map<Integer, byte[]> discoveryData() {
-        return discoData;
+    public DiscoveryDataPacket gridDiscoveryData() {
+        return dataPacket;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
index 80f4565..2c710b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
@@ -21,7 +21,7 @@ import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 
 /**
  * Sent by coordinator across the ring to finish node add process.
@@ -40,7 +40,7 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
      * TcpDiscoveryNodeAddFinishedMessage
      */
     @GridToStringExclude
-    private Map<UUID, Map<Integer, byte[]>> clientDiscoData;
+    private DiscoveryDataPacket clientDiscoData;
 
     /** */
     @GridToStringExclude
@@ -81,17 +81,17 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     /**
      * @return Discovery data for joined client.
      */
-    public Map<UUID, Map<Integer, byte[]>> clientDiscoData() {
+    public DiscoveryDataPacket clientDiscoData() {
         return clientDiscoData;
     }
 
     /**
      * @param clientDiscoData Discovery data for joined client.
      */
-    public void clientDiscoData(@Nullable Map<UUID, Map<Integer, byte[]>> clientDiscoData) {
+    public void clientDiscoData(DiscoveryDataPacket clientDiscoData) {
         this.clientDiscoData = clientDiscoData;
 
-        assert clientDiscoData == null || !clientDiscoData.containsKey(nodeId);
+        assert clientDiscoData == null || !clientDiscoData.hasDataFromNode(nodeId);
     }
 
     /**


[4/5] ignite git commit: IGNITE-4157 Use discovery custom messages instead of marshaller cache - Fixes #1271.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
index 18235d2..cb673d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoPolicy.java
@@ -40,14 +40,11 @@ public class GridIoPolicy {
     /** Utility cache execution pool. */
     public static final byte UTILITY_CACHE_POOL = 5;
 
-    /** Marshaller cache execution pool. */
-    public static final byte MARSH_CACHE_POOL = 6;
-
     /** IGFS pool. */
-    public static final byte IGFS_POOL = 7;
+    public static final byte IGFS_POOL = 6;
 
     /** Pool for handling distributed index range requests. */
-    public static final byte IDX_POOL = 8;
+    public static final byte IDX_POOL = 7;
 
     /** Data streamer execution pool. */
     public static final byte DATA_STREAMER_POOL = 9;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/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 9aa4db1..d15a87a 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.managers.discovery;
 
 import java.io.Externalizable;
-import java.io.Serializable;
 import java.lang.management.GarbageCollectorMXBean;
 import java.lang.management.ManagementFactory;
 import java.lang.management.MemoryMXBean;
@@ -104,6 +103,8 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.security.SecurityCredentials;
 import org.apache.ignite.plugin.segmentation.SegmentationPolicy;
 import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
@@ -642,41 +643,40 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         });
 
         spi.setDataExchange(new DiscoverySpiDataExchange() {
-            @Override public Map<Integer, Serializable> collect(UUID nodeId) {
-                assert nodeId != null;
+            @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
+                assert dataBag != null;
+                assert dataBag.joiningNodeId() != null;
 
-                Map<Integer, Serializable> data = new HashMap<>();
-
-                for (GridComponent comp : ctx.components()) {
-                    Serializable compData = comp.collectDiscoveryData(nodeId);
-
-                    if (compData != null) {
-                        assert comp.discoveryDataType() != null;
-
-                        data.put(comp.discoveryDataType().ordinal(), compData);
-                    }
+                if (ctx.localNodeId().equals(dataBag.joiningNodeId())) {
+                    for (GridComponent c : ctx.components())
+                        c.collectJoiningNodeData(dataBag);
+                }
+                else {
+                    for (GridComponent c : ctx.components())
+                        c.collectGridNodeData(dataBag);
                 }
 
-                return data;
+                return dataBag;
             }
 
-            @Override public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data) {
-                for (Map.Entry<Integer, Serializable> e : data.entrySet()) {
-                    GridComponent comp = null;
-
+            @Override public void onExchange(DiscoveryDataBag dataBag) {
+                if (ctx.localNodeId().equals(dataBag.joiningNodeId())) {
+                    //NodeAdded msg reached joining node after round-trip over the ring
                     for (GridComponent c : ctx.components()) {
-                        if (c.discoveryDataType() != null && c.discoveryDataType().ordinal() == e.getKey()) {
-                            comp = c;
-
-                            break;
-                        }
+                        if (c.discoveryDataType() != null)
+                            c.onGridDataReceived(dataBag.gridDiscoveryData(c.discoveryDataType().ordinal()));
                     }
+                }
+                else {
+                    //discovery data from newly joined node has to be applied to the current old node
+                    for (GridComponent c : ctx.components()) {
+                        if (c.discoveryDataType() != null) {
+                            JoiningNodeDiscoveryData data =
+                                    dataBag.newJoinerDiscoveryData(c.discoveryDataType().ordinal());
 
-                    if (comp != null)
-                        comp.onDiscoveryDataReceived(joiningNodeId, nodeId, e.getValue());
-                    else {
-                        if (log.isDebugEnabled())
-                            log.debug("Received discovery data for unknown component: " + e.getKey());
+                            if (data != null)
+                                c.onJoiningNodeDataReceived(data);
+                        }
                     }
                 }
             }
@@ -1555,6 +1555,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         return discoCache().allNodes();
     }
 
+    /** @return all alive server nodes is topology */
+    public Collection<ClusterNode> aliveSrvNodes() {
+        return discoCache().aliveSrvNodes();
+    }
+
     /** @return Full topology size. */
     public int size() {
         return discoCache().allNodes().size();
@@ -2538,6 +2543,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** Highest node order. */
         private final long maxOrder;
 
+        /** Alive server nodes */
+        private final Collection<ClusterNode> aliveSrvNodes;
+
         /**
          * Cached alive nodes list. As long as this collection doesn't accept {@code null}s use {@link
          * #maskNull(String)} before passing raw cache names to it.
@@ -2589,6 +2597,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             aliveSrvNodesWithCaches = new ConcurrentSkipListMap<>(GridNodeOrderComparator.INSTANCE);
             nodesByVer = new TreeMap<>();
 
+            List<ClusterNode> aliveSrvNodesList = new ArrayList<>(allNodes.size());
+
             long maxOrder0 = 0;
 
             Set<String> nearEnabledSet = new HashSet<>();
@@ -2640,8 +2650,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     }
                 }
 
-                if (hasCaches && alive(node.id()) && !CU.clientNode(node))
-                    aliveSrvNodesWithCaches.put(node, Boolean.TRUE);
+                if (alive(node.id()) && !CU.clientNode(node)) {
+                    aliveSrvNodesList.add(node);
+
+                    if (hasCaches)
+                        aliveSrvNodesWithCaches.put(node, Boolean.TRUE);
+                }
 
                 IgniteProductVersion nodeVer = U.productVersion(node);
 
@@ -2673,6 +2687,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             maxOrder = maxOrder0;
 
+            aliveSrvNodes = Collections.unmodifiableList(aliveSrvNodesList);
+
             allCacheNodes = Collections.unmodifiableMap(cacheMap);
             rmtCacheNodes = Collections.unmodifiableMap(rmtCacheMap);
             affCacheNodes = Collections.unmodifiableMap(dhtNodesMap);
@@ -2770,6 +2786,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
+         * Gets all alive server nodes.
+         */
+        Collection<ClusterNode> aliveSrvNodes() {
+            return aliveSrvNodes;
+        }
+
+        /**
          * Gets all remote nodes that have at least one cache configured.
          *
          * @param topVer Topology version.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
index e4896fd..4b4aec5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors;
 
-import java.io.Serializable;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
@@ -28,6 +26,9 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -79,12 +80,22 @@ public abstract class GridProcessorAdapter implements GridProcessor {
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
-        return null;
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheType.java
index 4886e61..c5855d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheType.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
 
@@ -38,12 +37,7 @@ public enum CacheType {
     /**
      * Internal replicated cache, should use separate thread pool.
      */
-    UTILITY(false, UTILITY_CACHE_POOL),
-
-    /**
-     * Internal marshaller cache, should use separate thread pool.
-     */
-    MARSHALLER(false, MARSH_CACHE_POOL);
+    UTILITY(false, UTILITY_CACHE_POOL);
 
     /** */
     private final boolean userCache;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index e414160..1bd7442 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -501,7 +501,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     @Override public final GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
         assert !CU.isUtilityCache(ctx.name());
         assert !CU.isAtomicsCache(ctx.name());
-        assert !CU.isMarshallerCache(ctx.name());
 
         CacheOperationContext opCtx = new CacheOperationContext(false, null, false, plc, false, null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 9487589..c5725e7 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -118,6 +118,9 @@ import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
@@ -138,6 +141,7 @@ import static org.apache.ignite.configuration.DeploymentMode.ISOLATED;
 import static org.apache.ignite.configuration.DeploymentMode.PRIVATE;
 import static org.apache.ignite.configuration.DeploymentMode.SHARED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC;
 import static org.apache.ignite.internal.IgniteComponentType.JTA;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG;
@@ -613,7 +617,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             ctx.config().getCacheStoreSessionListenerFactories()));
 
         for (int i = 0; i < cfgs.length; i++) {
-            if (ctx.config().isDaemon() && !CU.isMarshallerCache(cfgs[i].getName()))
+            if (ctx.config().isDaemon())
                 continue;
 
             cloneCheckSerializable(cfgs[i]);
@@ -644,8 +648,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (CU.isUtilityCache(cfg.getName()))
                 cacheType = CacheType.UTILITY;
-            else if (CU.isMarshallerCache(cfg.getName()))
-                cacheType = CacheType.MARSHALLER;
             else if (internalCaches.contains(maskNull(cfg.getName())))
                 cacheType = CacheType.INTERNAL;
             else
@@ -768,7 +770,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             // Start dynamic caches received from collect discovery data.
             for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                if (ctx.config().isDaemon() && !CU.isMarshallerCache(desc.cacheConfiguration().getName()))
+                if (ctx.config().isDaemon())
                     continue;
 
                 desc.clearRemoteConfigurations();
@@ -818,8 +820,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter<?, ?> cache : caches.values())
             onKernalStart(cache);
 
-        ctx.marshallerContext().onMarshallerCacheStarted(ctx);
-
         if (!ctx.config().isDaemon())
             ctx.cacheObjects().onUtilityCacheStarted();
 
@@ -845,7 +845,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         }
 
-        assert caches.containsKey(CU.MARSH_CACHE_NAME) : "Marshaller cache should be started";
         assert ctx.config().isDaemon() || caches.containsKey(CU.UTILITY_CACHE_NAME) : "Utility cache should be started";
     }
 
@@ -985,7 +984,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             boolean stopped;
 
-            boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
 
             if (!sysCache) {
                 DynamicCacheDescriptor oldDesc = cachesOnDisconnect.get(maskNull(name));
@@ -1890,12 +1889,24 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
-        return DiscoveryDataExchangeType.CACHE_PROC;
+        return CACHE_PROC;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        dataBag.addJoiningNodeData(CACHE_PROC.ordinal(), getDiscoveryData(dataBag.joiningNodeId()));
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Serializable collectDiscoveryData(UUID nodeId) {
-        boolean reconnect = ctx.localNodeId().equals(nodeId) && cachesOnDisconnect != null;
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        dataBag.addNodeSpecificData(CACHE_PROC.ordinal(), getDiscoveryData(dataBag.joiningNodeId()));
+    }
+
+    /**
+     * @param joiningNodeId Joining node id.
+     */
+    private Serializable getDiscoveryData(UUID joiningNodeId) {
+        boolean reconnect = ctx.localNodeId().equals(joiningNodeId) && cachesOnDisconnect != null;
 
         // Collect dynamically started caches to a single object.
         Collection<DynamicCacheChangeRequest> reqs;
@@ -1907,178 +1918,225 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             clientNodesMap = U.newHashMap(caches.size());
 
-            for (GridCacheAdapter<?, ?> cache : caches.values()) {
-                DynamicCacheDescriptor desc = cachesOnDisconnect.get(maskNull(cache.name()));
+            collectDataOnReconnectingNode(reqs, clientNodesMap, joiningNodeId);
+        }
+        else {
+            reqs = new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
 
-                if (desc == null)
-                    continue;
+            clientNodesMap = ctx.discovery().clientNodesMap();
 
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cache.name(), null);
+            collectDataOnGridNode(reqs);
+        }
 
-                req.startCacheConfiguration(desc.cacheConfiguration());
+        DynamicCacheChangeBatch batch = new DynamicCacheChangeBatch(reqs);
 
-                req.cacheType(desc.cacheType());
+        batch.clientNodes(clientNodesMap);
 
-                req.deploymentId(desc.deploymentId());
+        batch.clientReconnect(reconnect);
+
+        // Reset random batch ID so that serialized batches with the same descriptors will be exactly the same.
+        batch.id(null);
+
+        return batch;
+    }
 
-                req.receivedFrom(desc.receivedFrom());
+    /**
+     * @param reqs requests.
+     */
+    private void collectDataOnGridNode(Collection<DynamicCacheChangeRequest> reqs) {
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
-                reqs.add(req);
+            req.startCacheConfiguration(desc.cacheConfiguration());
 
-                Boolean nearEnabled = cache.isNear();
+            req.cacheType(desc.cacheType());
 
-                Map<UUID, Boolean> map = U.newHashMap(1);
+            req.deploymentId(desc.deploymentId());
 
-                map.put(nodeId, nearEnabled);
+            req.receivedFrom(desc.receivedFrom());
 
-                clientNodesMap.put(cache.name(), map);
-            }
+            reqs.add(req);
         }
-        else {
-            reqs = new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
 
-            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+        for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
-                req.startCacheConfiguration(desc.cacheConfiguration());
+            req.startCacheConfiguration(desc.cacheConfiguration());
 
-                req.cacheType(desc.cacheType());
+            req.template(true);
 
-                req.deploymentId(desc.deploymentId());
+            reqs.add(req);
+        }
+    }
 
-                req.receivedFrom(desc.receivedFrom());
+    /**
+     * @param reqs requests.
+     * @param clientNodesMap Client nodes map.
+     * @param nodeId Node id.
+     */
+    private void collectDataOnReconnectingNode(
+            Collection<DynamicCacheChangeRequest> reqs,
+            Map<String, Map<UUID, Boolean>> clientNodesMap,
+            UUID nodeId
+    ) {
+        for (GridCacheAdapter<?, ?> cache : caches.values()) {
+            DynamicCacheDescriptor desc = cachesOnDisconnect.get(maskNull(cache.name()));
 
-                reqs.add(req);
-            }
+            if (desc == null)
+                continue;
 
-            for (DynamicCacheDescriptor desc : registeredTemplates.values()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cache.name(), null);
 
-                req.startCacheConfiguration(desc.cacheConfiguration());
+            req.startCacheConfiguration(desc.cacheConfiguration());
 
-                req.template(true);
+            req.cacheType(desc.cacheType());
 
-                reqs.add(req);
-            }
+            req.deploymentId(desc.deploymentId());
 
-            clientNodesMap = ctx.discovery().clientNodesMap();
-        }
+            req.receivedFrom(desc.receivedFrom());
 
-        DynamicCacheChangeBatch batch = new DynamicCacheChangeBatch(reqs);
+            reqs.add(req);
 
-        batch.clientNodes(clientNodesMap);
+            Boolean nearEnabled = cache.isNear();
 
-        batch.clientReconnect(reconnect);
+            Map<UUID, Boolean> map = U.newHashMap(1);
 
-        // Reset random batch ID so that serialized batches with the same descriptors will be exactly the same.
-        batch.id(null);
+            map.put(nodeId, nearEnabled);
 
-        return batch;
+            clientNodesMap.put(cache.name(), map);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
-        if (data instanceof DynamicCacheChangeBatch) {
-            DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)data;
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
+        if (data.hasJoiningNodeData()) {
+            Serializable joiningNodeData = data.joiningNodeData();
+            if (joiningNodeData instanceof DynamicCacheChangeBatch)
+                onDiscoDataReceived(
+                        data.joiningNodeId(),
+                        data.joiningNodeId(),
+                        (DynamicCacheChangeBatch) joiningNodeData);
+        }
+    }
 
-            if (batch.clientReconnect()) {
-                if (ctx.clientDisconnected()) {
-                    if (clientReconnectReqs == null)
-                        clientReconnectReqs = new LinkedHashMap<>();
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
 
-                    clientReconnectReqs.put(joiningNodeId, batch);
+        if (nodeSpecData != null) {
+            for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet()) {
+                if (e.getValue() != null && e.getValue() instanceof DynamicCacheChangeBatch) {
+                    DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch) e.getValue();
 
-                    return;
+                    onDiscoDataReceived(data.joiningNodeId(), e.getKey(), batch);
                 }
+            }
+        }
+    }
 
-                processClientReconnectData(joiningNodeId, batch);
+    /**
+     * @param joiningNodeId Joining node id.
+     * @param rmtNodeId Rmt node id.
+     * @param batch Batch.
+     */
+    private void onDiscoDataReceived(UUID joiningNodeId, UUID rmtNodeId, DynamicCacheChangeBatch batch) {
+        if (batch.clientReconnect()) {
+            if (ctx.clientDisconnected()) {
+                if (clientReconnectReqs == null)
+                    clientReconnectReqs = new LinkedHashMap<>();
+
+                clientReconnectReqs.put(joiningNodeId, batch);
+
+                return;
             }
-            else {
-                for (DynamicCacheChangeRequest req : batch.requests()) {
-                    initReceivedCacheConfiguration(req);
 
-                    if (req.template()) {
-                        CacheConfiguration ccfg = req.startCacheConfiguration();
+            processClientReconnectData(joiningNodeId, batch);
+        }
+        else {
+            for (DynamicCacheChangeRequest req : batch.requests()) {
+                initReceivedCacheConfiguration(req);
+
+                if (req.template()) {
+                    CacheConfiguration ccfg = req.startCacheConfiguration();
 
-                        assert ccfg != null : req;
+                    assert ccfg != null : req;
 
-                        DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
+                    DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
 
-                        if (existing == null) {
-                            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                    if (existing == null) {
+                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
                                 ctx,
                                 ccfg,
                                 req.cacheType(),
                                 true,
                                 req.deploymentId());
 
-                            registeredTemplates.put(maskNull(req.cacheName()), desc);
-                        }
-
-                        continue;
+                        registeredTemplates.put(maskNull(req.cacheName()), desc);
                     }
 
-                    DynamicCacheDescriptor existing = registeredCaches.get(maskNull(req.cacheName()));
+                    continue;
+                }
 
-                    if (req.start() && !req.clientStartOnly()) {
-                        CacheConfiguration ccfg = req.startCacheConfiguration();
+                DynamicCacheDescriptor existing = registeredCaches.get(maskNull(req.cacheName()));
 
-                        if (existing != null) {
-                            if (joiningNodeId.equals(ctx.localNodeId())) {
-                                existing.receivedFrom(req.receivedFrom());
+                if (req.start() && !req.clientStartOnly()) {
+                    CacheConfiguration ccfg = req.startCacheConfiguration();
 
-                                existing.deploymentId(req.deploymentId());
-                            }
+                    if (existing != null) {
+                        if (joiningNodeId.equals(ctx.localNodeId())) {
+                            existing.receivedFrom(req.receivedFrom());
 
-                            if (existing.locallyConfigured()) {
-                                existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration());
+                            existing.deploymentId(req.deploymentId());
+                        }
 
-                                ctx.discovery().setCacheFilter(
+                        if (existing.locallyConfigured()) {
+                            existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration());
+
+                            ctx.discovery().setCacheFilter(
                                     req.cacheName(),
                                     ccfg.getNodeFilter(),
                                     ccfg.getNearConfiguration() != null,
                                     ccfg.getCacheMode());
-                            }
                         }
-                        else {
-                            assert req.cacheType() != null : req;
+                    }
+                    else {
+                        assert req.cacheType() != null : req;
 
-                            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
                                 ctx,
                                 ccfg,
                                 req.cacheType(),
                                 false,
                                 req.deploymentId());
 
-                            // Received statically configured cache.
-                            if (req.initiatingNodeId() == null)
-                                desc.staticallyConfigured(true);
+                        // Received statically configured cache.
+                        if (req.initiatingNodeId() == null)
+                            desc.staticallyConfigured(true);
 
-                            if (joiningNodeId.equals(ctx.localNodeId()))
-                                desc.receivedOnDiscovery(true);
+                        if (joiningNodeId.equals(ctx.localNodeId()))
+                            desc.receivedOnDiscovery(true);
 
-                            desc.receivedFrom(req.receivedFrom());
+                        desc.receivedFrom(req.receivedFrom());
 
-                            DynamicCacheDescriptor old = registeredCaches.put(maskNull(req.cacheName()), desc);
+                        DynamicCacheDescriptor old = registeredCaches.put(maskNull(req.cacheName()), desc);
 
-                            assert old == null : old;
+                        assert old == null : old;
 
-                            ctx.discovery().setCacheFilter(
+                        ctx.discovery().setCacheFilter(
                                 req.cacheName(),
                                 ccfg.getNodeFilter(),
                                 ccfg.getNearConfiguration() != null,
                                 ccfg.getCacheMode());
-                        }
                     }
                 }
+            }
 
-                if (!F.isEmpty(batch.clientNodes())) {
-                    for (Map.Entry<String, Map<UUID, Boolean>> entry : batch.clientNodes().entrySet()) {
-                        String cacheName = entry.getKey();
+            if (!F.isEmpty(batch.clientNodes())) {
+                for (Map.Entry<String, Map<UUID, Boolean>> entry : batch.clientNodes().entrySet()) {
+                    String cacheName = entry.getKey();
 
-                        for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
-                            ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
-                    }
+                    for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
+                        ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
                 }
             }
         }
@@ -2098,7 +2156,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             String name = req.cacheName();
 
-            boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+            boolean sysCache = CU.isUtilityCache(name) || CU.isAtomicsCache(name);
 
             if (!sysCache) {
                 DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
@@ -3127,13 +3185,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @return Marshaller system cache.
-     */
-    public GridCacheAdapter<Integer, String> marshallerCache() {
-        return internalCache(CU.MARSH_CACHE_NAME);
-    }
-
-    /**
      * Gets utility cache.
      *
      * @return Utility cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 0f855fe..e694e30 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -43,7 +43,6 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
         boolean cleanupDisabled = cctx.kernalContext().isDaemon() ||
             !cctx.config().isEagerTtl() ||
             CU.isAtomicsCache(cctx.name()) ||
-            CU.isMarshallerCache(cctx.name()) ||
             CU.isUtilityCache(cctx.name()) ||
             (cctx.kernalContext().clientNode() && cctx.config().getNearConfiguration() == null);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 969c41a..61a57f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -154,9 +154,6 @@ public class GridCacheUtils {
     /** Atomics system cache name. */
     public static final String ATOMICS_CACHE_NAME = "ignite-atomics-sys-cache";
 
-    /** Marshaller system cache name. */
-    public static final String MARSH_CACHE_NAME = "ignite-marshaller-sys-cache";
-
     /** */
     public static final String CACHE_MSG_LOG_CATEGORY = "org.apache.ignite.cache.msg";
 
@@ -1176,14 +1173,6 @@ public class GridCacheUtils {
 
     /**
      * @param cacheName Cache name.
-     * @return {@code True} if this is marshaller system cache.
-     */
-    public static boolean isMarshallerCache(String cacheName) {
-        return MARSH_CACHE_NAME.equals(cacheName);
-    }
-
-    /**
-     * @param cacheName Cache name.
      * @return {@code True} if this is utility system cache.
      */
     public static boolean isUtilityCache(String cacheName) {
@@ -1203,7 +1192,7 @@ public class GridCacheUtils {
      * @return {@code True} if system cache.
      */
     public static boolean isSystemCache(String cacheName) {
-        return isMarshallerCache(cacheName) || isUtilityCache(cacheName) || isHadoopSystemCache(cacheName) ||
+        return isUtilityCache(cacheName) || isHadoopSystemCache(cacheName) ||
             isAtomicsCache(cacheName);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
index a00cf3e..976f05f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFutureAdapter.java
@@ -62,6 +62,19 @@ public abstract class GridNearOptimisticTxPrepareFutureAdapter extends GridNearT
         }
 
         if (topVer != null) {
+            try {
+                IgniteCheckedException err = tx.txState().validateTopology(cctx, topologyReadLock());
+
+                if (err != null) {
+                    onDone(err);
+
+                    return;
+                }
+            }
+            finally {
+                topologyReadUnlock();
+            }
+
             tx.topologyVersion(topVer);
 
             cctx.mvcc().addFuture(this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 85c01d9..a0ab0be 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -2367,7 +2367,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 },
                 new P1<IgniteInternalCache<?, ?>>() {
                     @Override public boolean apply(IgniteInternalCache<?, ?> c) {
-                        return !CU.MARSH_CACHE_NAME.equals(c.name()) && !CU.UTILITY_CACHE_NAME.equals(c.name()) &&
+                        return !CU.UTILITY_CACHE_NAME.equals(c.name()) &&
                             !CU.ATOMICS_CACHE_NAME.equals(c.name());
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index 6500cf3..d1c8b2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cluster;
 
 import java.io.Serializable;
 import java.lang.ref.WeakReference;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Timer;
@@ -36,9 +37,12 @@ import org.apache.ignite.internal.util.GridTimerTask;
 import org.apache.ignite.internal.util.future.IgniteFinishedFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CLUSTER_PROC;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER_STR;
 
 /**
@@ -98,11 +102,24 @@ public class ClusterProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
-        return DiscoveryDataExchangeType.CLUSTER_PROC;
+        return CLUSTER_PROC;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Serializable collectDiscoveryData(UUID nodeId) {
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        dataBag.addJoiningNodeData(CLUSTER_PROC.ordinal(), getDiscoveryData());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        dataBag.addNodeSpecificData(CLUSTER_PROC.ordinal(), getDiscoveryData());
+    }
+
+
+    /**
+     * @return Discovery data.
+     */
+    private Serializable getDiscoveryData() {
         HashMap<String, Object> map = new HashMap<>();
 
         map.put(ATTR_UPDATE_NOTIFIER_STATUS, notifyEnabled.get());
@@ -111,16 +128,36 @@ public class ClusterProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
-        if (joiningNodeId.equals(ctx.localNodeId())) {
-            Map<String, Object> map = (Map<String, Object>)data;
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
 
-            if (map != null && map.containsKey(ATTR_UPDATE_NOTIFIER_STATUS))
-                notifyEnabled.set((Boolean)map.get(ATTR_UPDATE_NOTIFIER_STATUS));
+        if (nodeSpecData != null) {
+            Boolean lstFlag = findLastFlag(nodeSpecData.values());
+
+            if (lstFlag != null)
+                notifyEnabled.set(lstFlag);
         }
     }
 
+
+    /**
+     * @param vals collection to seek through.
+     */
+    private Boolean findLastFlag(Collection<Serializable> vals) {
+        Boolean flag = null;
+
+        for (Serializable ser : vals) {
+            if (ser != null) {
+                Map<String, Object> map = (Map<String, Object>) ser;
+
+                if (map.containsKey(ATTR_UPDATE_NOTIFIER_STATUS))
+                    flag = (Boolean) map.get(ATTR_UPDATE_NOTIFIER_STATUS);
+            }
+        }
+
+        return flag;
+    }
+
     /** {@inheritDoc} */
     @Override public void onKernalStart() throws IgniteCheckedException {
         if (notifyEnabled.get()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 9fd9b6d..b0db510 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -78,6 +78,9 @@ import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -85,6 +88,7 @@ import org.jsr166.ConcurrentHashMap8;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_NODE_SEGMENTED;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CONTINUOUS_PROC;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CONTINUOUS;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
@@ -319,8 +323,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             }
         });
 
-        ctx.marshallerContext().onContinuousProcessorStarted(ctx);
-
         ctx.cacheObjects().onContinuousProcessorStarted(ctx);
 
         ctx.service().onContinuousProcessorStarted(ctx);
@@ -403,36 +405,42 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
-        return DiscoveryDataExchangeType.CONTINUOUS_PROC;
+        return CONTINUOUS_PROC;
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
-        if (log.isDebugEnabled()) {
-            log.debug("collectDiscoveryData [node=" + nodeId +
-                ", loc=" + ctx.localNodeId() +
-                ", locInfos=" + locInfos +
-                ", clientInfos=" + clientInfos +
-                ']');
-        }
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        Serializable data = getDiscoveryData(dataBag.joiningNodeId());
 
-        if (!nodeId.equals(ctx.localNodeId()) || !locInfos.isEmpty()) {
-            Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos0 = U.newHashMap(clientInfos.size());
+        if (data != null)
+            dataBag.addJoiningNodeData(CONTINUOUS_PROC.ordinal(), data);
+    }
 
-            for (Map.Entry<UUID, Map<UUID, LocalRoutineInfo>> e : clientInfos.entrySet()) {
-                Map<UUID, LocalRoutineInfo> cp = U.newHashMap(e.getValue().size());
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        Serializable data = getDiscoveryData(dataBag.joiningNodeId());
 
-                for (Map.Entry<UUID, LocalRoutineInfo> e0 : e.getValue().entrySet())
-                    cp.put(e0.getKey(), e0.getValue());
+        if (data != null)
+            dataBag.addNodeSpecificData(CONTINUOUS_PROC.ordinal(), data);
+    }
 
-                clientInfos0.put(e.getKey(), cp);
-            }
+    /**
+     * @param joiningNodeId Joining node id.
+     */
+    private Serializable getDiscoveryData(UUID joiningNodeId) {
+        if (log.isDebugEnabled()) {
+            log.debug("collectDiscoveryData [node=" + joiningNodeId +
+                    ", loc=" + ctx.localNodeId() +
+                    ", locInfos=" + locInfos +
+                    ", clientInfos=" + clientInfos +
+                    ']');
+        }
 
-            if (nodeId.equals(ctx.localNodeId()) && ctx.discovery().localNode().isClient()) {
-                Map<UUID, LocalRoutineInfo> infos = new HashMap<>();
+        if (!joiningNodeId.equals(ctx.localNodeId()) || !locInfos.isEmpty()) {
+            Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos0 = copyClientInfos(clientInfos);
 
-                for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet())
-                    infos.put(e.getKey(), e.getValue());
+            if (joiningNodeId.equals(ctx.localNodeId()) && ctx.discovery().localNode().isClient()) {
+                Map<UUID, LocalRoutineInfo> infos = copyLocalInfos(locInfos);
 
                 clientInfos0.put(ctx.localNodeId(), infos);
             }
@@ -445,31 +453,75 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 LocalRoutineInfo info = e.getValue();
 
                 data.addItem(new DiscoveryDataItem(routineId,
-                    info.prjPred,
-                    info.hnd,
-                    info.bufSize,
-                    info.interval,
-                    info.autoUnsubscribe));
+                        info.prjPred,
+                        info.hnd,
+                        info.bufSize,
+                        info.interval,
+                        info.autoUnsubscribe));
             }
 
             return data;
         }
-
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable obj) {
-        DiscoveryData data = (DiscoveryData)obj;
+    /**
+     * @param clientInfos Client infos.
+     */
+    private Map<UUID, Map<UUID, LocalRoutineInfo>> copyClientInfos(Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos) {
+        Map<UUID, Map<UUID, LocalRoutineInfo>> res = U.newHashMap(clientInfos.size());
+
+        for (Map.Entry<UUID, Map<UUID, LocalRoutineInfo>> e : clientInfos.entrySet()) {
+            Map<UUID, LocalRoutineInfo> cp = U.newHashMap(e.getValue().size());
+
+            for (Map.Entry<UUID, LocalRoutineInfo> e0 : e.getValue().entrySet())
+                cp.put(e0.getKey(), e0.getValue());
+
+            res.put(e.getKey(), cp);
+        }
 
+        return res;
+    }
+
+    /**
+     * @param locInfos Locale infos.
+     */
+    private Map<UUID, LocalRoutineInfo> copyLocalInfos(Map<UUID, LocalRoutineInfo> locInfos) {
+        Map<UUID, LocalRoutineInfo> res = U.newHashMap(locInfos.size());
+
+        for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet())
+            res.put(e.getKey(), e.getValue());
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
         if (log.isDebugEnabled()) {
-            log.info("onDiscoveryDataReceived [joining=" + joiningNodeId +
-                ", rmtNodeId=" + rmtNodeId +
-                ", loc=" + ctx.localNodeId() +
-                ", data=" + data +
-                ']');
+            log.info("onJoiningNodeDataReceived [joining=" + data.joiningNodeId() +
+                    ", loc=" + ctx.localNodeId() +
+                    ", data=" + data.joiningNodeData() +
+                    ']');
         }
 
+        if (data.hasJoiningNodeData())
+            onDiscoDataReceived((DiscoveryData) data.joiningNodeData());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        Map<UUID, Serializable> nodeSpecData = data.nodeSpecificData();
+
+        if (nodeSpecData != null) {
+            for (Map.Entry<UUID, Serializable> e : nodeSpecData.entrySet())
+                onDiscoDataReceived((DiscoveryData) e.getValue());
+        }
+    }
+
+    /**
+     * @param data received discovery data.
+     */
+    private void onDiscoDataReceived(DiscoveryData data) {
         if (!ctx.isDaemon() && data != null) {
             for (DiscoveryDataItem item : data.items) {
                 try {
@@ -478,14 +530,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                     // Register handler only if local node passes projection predicate.
                     if ((item.prjPred == null || item.prjPred.apply(ctx.discovery().localNode())) &&
-                        !locInfos.containsKey(item.routineId))
+                            !locInfos.containsKey(item.routineId))
                         registerHandler(data.nodeId, item.routineId, item.hnd, item.bufSize, item.interval,
-                            item.autoUnsubscribe, false);
+                                item.autoUnsubscribe, false);
 
                     if (!item.autoUnsubscribe)
                         // Register routine locally.
                         locInfos.putIfAbsent(item.routineId, new LocalRoutineInfo(
-                            item.prjPred, item.hnd, item.bufSize, item.interval, item.autoUnsubscribe));
+                                item.prjPred, item.hnd, item.bufSize, item.interval, item.autoUnsubscribe));
                 }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to register continuous handler.", e);
@@ -508,12 +560,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
                             if (info.prjPred == null || info.prjPred.apply(ctx.discovery().localNode())) {
                                 registerHandler(clientNodeId,
-                                    routineId,
-                                    info.hnd,
-                                    info.bufSize,
-                                    info.interval,
-                                    info.autoUnsubscribe,
-                                    false);
+                                        routineId,
+                                        info.hnd,
+                                        info.bufSize,
+                                        info.interval,
+                                        info.autoUnsubscribe,
+                                        false);
                             }
                         }
                         catch (IgniteCheckedException err) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/ClientRequestFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/ClientRequestFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/ClientRequestFuture.java
new file mode 100644
index 0000000..a361760
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/ClientRequestFuture.java
@@ -0,0 +1,183 @@
+/*
+ * 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.processors.marshaller;
+
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Queue;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Future responsible for requesting missing marshaller mapping from one of available server nodes.
+ *
+ * Handles scenarios when server nodes leave cluster. If node that was requested for mapping leaves the cluster or fails,
+ * mapping is automatically requested from the next node available in topology.
+ */
+final class ClientRequestFuture extends GridFutureAdapter<MappingExchangeResult> {
+    /** */
+    private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
+
+    /** */
+    private static IgniteLogger log;
+
+    /** */
+    private final GridIoManager ioMgr;
+
+    /** */
+    private final GridDiscoveryManager discoMgr;
+
+    /** */
+    private final MarshallerMappingItem item;
+
+    /** */
+    private final Map<MarshallerMappingItem, ClientRequestFuture> syncMap;
+
+    /** */
+    private final Queue<ClusterNode> aliveSrvNodes;
+
+    /** */
+    private ClusterNode pendingNode;
+
+    /**
+     * @param ctx Context.
+     * @param item Item.
+     * @param syncMap Sync map.
+     */
+    ClientRequestFuture(
+            GridKernalContext ctx,
+            MarshallerMappingItem item,
+            Map<MarshallerMappingItem, ClientRequestFuture> syncMap
+    ) {
+        ioMgr = ctx.io();
+        discoMgr = ctx.discovery();
+        aliveSrvNodes = new LinkedList<>(discoMgr.aliveSrvNodes());
+        this.item = item;
+        this.syncMap = syncMap;
+
+        if (log == null)
+            log = U.logger(ctx, logRef, ClientRequestFuture.class);
+    }
+
+    /**
+     *
+     */
+    void requestMapping() {
+        boolean noSrvsInCluster;
+
+        synchronized (this) {
+            while (!aliveSrvNodes.isEmpty()) {
+                ClusterNode srvNode = aliveSrvNodes.poll();
+
+                try {
+                    ioMgr.send(
+                            srvNode,
+                            GridTopic.TOPIC_MAPPING_MARSH,
+                            new MissingMappingRequestMessage(
+                                    item.platformId(),
+                                    item.typeId()),
+                            GridIoPolicy.SYSTEM_POOL);
+
+                    if (discoMgr.node(srvNode.id()) == null)
+                        continue;
+
+                    pendingNode = srvNode;
+
+                    break;
+                }
+                catch (IgniteCheckedException ignored) {
+                    U.warn(log,
+                            "Failed to request marshaller mapping from remote node (proceeding with the next one): "
+                                    + srvNode);
+                }
+            }
+
+            noSrvsInCluster = pendingNode == null;
+        }
+
+        if (noSrvsInCluster)
+            onDone(MappingExchangeResult.createFailureResult(
+                    new IgniteCheckedException(
+                            "All server nodes have left grid, cannot request mapping [platformId: "
+                                    + item.platformId()
+                                    + "; typeId: "
+                                    + item.typeId() + "]")));
+    }
+
+    /**
+     * @param nodeId Node ID.
+     * @param res Mapping Request Result.
+     */
+    void onResponse(UUID nodeId, MappingExchangeResult res) {
+        MappingExchangeResult res0 = null;
+
+        synchronized (this) {
+            if (pendingNode != null && pendingNode.id().equals(nodeId))
+                res0 = res;
+        }
+
+        if (res0 != null)
+            onDone(res0);
+    }
+
+    /**
+     * If left node is actually the one latest mapping request was sent to,
+     * request is sent again to the next node in topology.
+     *
+     * @param leftNodeId Left node id.
+     */
+    void onNodeLeft(UUID leftNodeId) {
+        boolean reqAgain = false;
+
+        synchronized (this) {
+            if (pendingNode != null && pendingNode.id().equals(leftNodeId)) {
+                aliveSrvNodes.remove(pendingNode);
+
+                pendingNode = null;
+
+                reqAgain = true;
+            }
+        }
+
+        if (reqAgain)
+            requestMapping();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean onDone(@Nullable MappingExchangeResult res, @Nullable Throwable err) {
+        assert res != null;
+
+        boolean done = super.onDone(res, err);
+
+        if (done)
+            syncMap.remove(item);
+
+        return done;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
new file mode 100644
index 0000000..7356e6c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/GridMarshallerMappingProcessor.java
@@ -0,0 +1,326 @@
+/*
+ * 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.processors.marshaller;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteClientDisconnectedCheckedException;
+import org.apache.ignite.internal.MarshallerContextImpl;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.managers.discovery.CustomEventListener;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
+import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.MARSHALLER_PROC;
+import static org.apache.ignite.internal.GridTopic.TOPIC_MAPPING_MARSH;
+import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
+
+/**
+ * Processor responsible for managing custom {@link DiscoveryCustomMessage}
+ * events for exchanging marshalling mappings between nodes in grid.
+ *
+ * In particular it processes two flows:
+ * <ul>
+ *     <li>
+ *         Some node, server or client, wants to add new mapping for some class.
+ *         In that case a pair of {@link MappingProposedMessage} and {@link MappingAcceptedMessage} events is used.
+ *     </li>
+ *     <li>
+ *         As discovery events are delivered to clients asynchronously,
+ *         client node may not have some mapping when server nodes in the grid are already allowed to use the mapping.
+ *         In that situation client sends a {@link MissingMappingRequestMessage} request
+ *         and processor handles it as well as {@link MissingMappingResponseMessage} message.
+ *     </li>
+ * </ul>
+ */
+public class GridMarshallerMappingProcessor extends GridProcessorAdapter {
+    /** */
+    private final MarshallerContextImpl marshallerCtx;
+
+    /** */
+    private final ConcurrentMap<MarshallerMappingItem, GridFutureAdapter<MappingExchangeResult>> mappingExchangeSyncMap
+            = new ConcurrentHashMap8<>();
+
+    /** */
+    private final ConcurrentMap<MarshallerMappingItem, ClientRequestFuture> clientReqSyncMap = new ConcurrentHashMap8<>();
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public GridMarshallerMappingProcessor(GridKernalContext ctx) {
+        super(ctx);
+
+        marshallerCtx = ctx.marshallerContext();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        GridDiscoveryManager discoMgr = ctx.discovery();
+        GridIoManager ioMgr = ctx.io();
+
+        MarshallerMappingTransport transport = new MarshallerMappingTransport(
+                ctx,
+                mappingExchangeSyncMap,
+                clientReqSyncMap
+        );
+        marshallerCtx.onMarshallerProcessorStarted(ctx, transport);
+
+        discoMgr.setCustomEventListener(MappingProposedMessage.class, new MarshallerMappingExchangeListener());
+
+        discoMgr.setCustomEventListener(MappingAcceptedMessage.class, new MappingAcceptedListener());
+
+        if (!ctx.clientNode())
+            ioMgr.addMessageListener(TOPIC_MAPPING_MARSH, new MissingMappingRequestListener(ioMgr));
+        else
+            ioMgr.addMessageListener(TOPIC_MAPPING_MARSH, new MissingMappingResponseListener());
+
+        if (ctx.clientNode())
+            ctx.event().addLocalEventListener(new GridLocalEventListener() {
+                @Override public void onEvent(Event evt) {
+                    DiscoveryEvent evt0 = (DiscoveryEvent) evt;
+
+                    if (!ctx.isStopping()) {
+                        for (ClientRequestFuture fut : clientReqSyncMap.values())
+                            fut.onNodeLeft(evt0.eventNode().id());
+                    }
+                }
+            }, EVT_NODE_LEFT, EVT_NODE_FAILED);
+    }
+
+    /**
+     *
+     */
+    private final class MissingMappingRequestListener implements GridMessageListener {
+        /** */
+        private final GridIoManager ioMgr;
+
+        /**
+         * @param ioMgr Io manager.
+         */
+        MissingMappingRequestListener(GridIoManager ioMgr) {
+            this.ioMgr = ioMgr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessage(UUID nodeId, Object msg) {
+            assert msg instanceof MissingMappingRequestMessage : msg;
+
+            MissingMappingRequestMessage msg0 = (MissingMappingRequestMessage) msg;
+
+            byte platformId = msg0.platformId();
+            int typeId = msg0.typeId();
+
+            String resolvedClsName = marshallerCtx.resolveMissedMapping(platformId, typeId);
+
+            try {
+                ioMgr.send(
+                        nodeId,
+                        TOPIC_MAPPING_MARSH,
+                        new MissingMappingResponseMessage(platformId, typeId, resolvedClsName),
+                        SYSTEM_POOL);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send missing mapping response.", e);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private final class MissingMappingResponseListener implements GridMessageListener {
+        /** {@inheritDoc} */
+        @Override public void onMessage(UUID nodeId, Object msg) {
+            assert msg instanceof MissingMappingResponseMessage : msg;
+
+            MissingMappingResponseMessage msg0 = (MissingMappingResponseMessage) msg;
+
+            byte platformId = msg0.platformId();
+            int typeId = msg0.typeId();
+            String resolvedClsName = msg0.className();
+
+            MarshallerMappingItem item = new MarshallerMappingItem(platformId, typeId, null);
+
+            GridFutureAdapter<MappingExchangeResult> fut = clientReqSyncMap.get(item);
+
+            if (fut != null) {
+                if (resolvedClsName != null) {
+                    marshallerCtx.onMissedMappingResolved(item, resolvedClsName);
+
+                    fut.onDone(MappingExchangeResult.createSuccessfulResult(resolvedClsName));
+                }
+                else
+                    fut.onDone(MappingExchangeResult.createFailureResult(
+                            new IgniteCheckedException(
+                                    "Failed to resolve mapping [platformId: "
+                                            + platformId
+                                            + ", typeId: "
+                                            + typeId + "]")));
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private final class MarshallerMappingExchangeListener implements CustomEventListener<MappingProposedMessage> {
+        /** {@inheritDoc} */
+        @Override public void onCustomEvent(
+                AffinityTopologyVersion topVer,
+                ClusterNode snd,
+                MappingProposedMessage msg
+        ) {
+            if (!ctx.isStopping()) {
+                if (msg.duplicated())
+                    return;
+
+                if (!msg.inConflict()) {
+                    MarshallerMappingItem item = msg.mappingItem();
+                    String conflictingName = marshallerCtx.onMappingProposed(item);
+
+                    if (conflictingName != null) {
+                        if (conflictingName.equals(item.className()))
+                            msg.markDuplicated();
+                        else
+                            msg.conflictingWithClass(conflictingName);
+                    }
+                }
+                else {
+                    UUID origNodeId = msg.origNodeId();
+
+                    if (origNodeId.equals(ctx.localNodeId())) {
+                        GridFutureAdapter<MappingExchangeResult> fut = mappingExchangeSyncMap.get(msg.mappingItem());
+
+                        assert fut != null: msg;
+
+                        fut.onDone(MappingExchangeResult.createFailureResult(
+                                duplicateMappingException(msg.mappingItem(), msg.conflictingClassName())));
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param mappingItem Mapping item.
+         * @param conflictingClsName Conflicting class name.
+         */
+        private IgniteCheckedException duplicateMappingException(
+                MarshallerMappingItem mappingItem,
+                String conflictingClsName
+        ) {
+            return new IgniteCheckedException("Duplicate ID [platformId="
+                    + mappingItem.platformId()
+                    + ", typeId="
+                    + mappingItem.typeId()
+                    + ", oldCls="
+                    + conflictingClsName
+                    + ", newCls="
+                    + mappingItem.className() + "]");
+        }
+    }
+
+    /**
+     *
+     */
+    private final class MappingAcceptedListener implements CustomEventListener<MappingAcceptedMessage> {
+        /** {@inheritDoc} */
+        @Override public void onCustomEvent(
+                AffinityTopologyVersion topVer,
+                ClusterNode snd,
+                MappingAcceptedMessage msg
+        ) {
+            MarshallerMappingItem item = msg.getMappingItem();
+            marshallerCtx.onMappingAccepted(item);
+
+            GridFutureAdapter<MappingExchangeResult> fut = mappingExchangeSyncMap.get(item);
+
+            if (fut != null)
+                fut.onDone(MappingExchangeResult.createSuccessfulResult(item.className()));
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        if (!dataBag.commonDataCollectedFor(MARSHALLER_PROC.ordinal()))
+            dataBag.addGridCommonData(MARSHALLER_PROC.ordinal(), marshallerCtx.getCachedMappings());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        List<Map<Integer, MappedName>> mappings = (List<Map<Integer, MappedName>>) data.commonData();
+
+        if (mappings != null) {
+            for (int i = 0; i < mappings.size(); i++) {
+                Map<Integer, MappedName> map;
+
+                if ((map = mappings.get(i)) != null)
+                    marshallerCtx.onMappingDataReceived((byte) i, map);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        cancelFutures(MappingExchangeResult.createFailureResult(new IgniteClientDisconnectedCheckedException(
+                ctx.cluster().clientReconnectFuture(),
+                "Failed to propose or request mapping, client node disconnected.")));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        marshallerCtx.onMarshallerProcessorStop();
+
+        cancelFutures(MappingExchangeResult.createExchangeDisabledResult());
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
+        return MARSHALLER_PROC;
+    }
+
+    /**
+     * @param res Response.
+     */
+    private void cancelFutures(MappingExchangeResult res) {
+        for (GridFutureAdapter<MappingExchangeResult> fut : mappingExchangeSyncMap.values())
+            fut.onDone(res);
+
+        for (GridFutureAdapter<MappingExchangeResult> fut : clientReqSyncMap.values())
+            fut.onDone(res);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappedName.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappedName.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappedName.java
new file mode 100644
index 0000000..c13c48e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappedName.java
@@ -0,0 +1,63 @@
+/*
+ * 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.processors.marshaller;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Contains mapped class name and boolean flag showing whether this mapping was accepted by other nodes or not.
+ */
+public final class MappedName implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final String clsName;
+
+    /** */
+    private final boolean accepted;
+
+    /**
+     * @param clsName Class name.
+     * @param accepted Accepted.
+     */
+    public MappedName(String clsName, boolean accepted) {
+        this.clsName = clsName;
+        this.accepted = accepted;
+    }
+
+    /**
+     *
+     */
+    public String className() {
+        return clsName;
+    }
+
+    /**
+     *
+     */
+    public boolean accepted() {
+        return accepted;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MappedName.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
new file mode 100644
index 0000000..23c2858
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingAcceptedMessage.java
@@ -0,0 +1,71 @@
+/*
+ * 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.processors.marshaller;
+
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Is sent as an acknowledgement for successfully proposed new mapping (see {@link MappingProposedMessage}).
+ *
+ * If any nodes were waiting for this mapping to be accepted they will be unblocked on receiving this message.
+ */
+public class MappingAcceptedMessage implements DiscoveryCustomMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** */
+    private final MarshallerMappingItem item;
+
+    /**
+     * @param item Item.
+     */
+    MappingAcceptedMessage(MarshallerMappingItem item) {
+        this.item = item;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** */
+    MarshallerMappingItem getMappingItem() {
+        return item;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MappingAcceptedMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingExchangeResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingExchangeResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingExchangeResult.java
new file mode 100644
index 0000000..4bc1442
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingExchangeResult.java
@@ -0,0 +1,96 @@
+/*
+ * 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.processors.marshaller;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ *
+ */
+public class MappingExchangeResult {
+    /**  */
+    private final String acceptedClsName;
+
+    /** */
+    private final IgniteCheckedException error;
+
+    /** */
+    private final ResultType resType;
+
+    /** */
+    private enum ResultType {
+        /** */
+        SUCCESS,
+
+        /** */
+        FAILURE,
+
+        /** */
+        EXCHANGE_DISABLED
+    }
+
+    /**
+     */
+    private MappingExchangeResult(ResultType resType, String acceptedClsName, IgniteCheckedException error) {
+        this.resType = resType;
+        this.acceptedClsName = acceptedClsName;
+        this.error = error;
+    }
+
+    /**  */
+    public String className() {
+        return acceptedClsName;
+    }
+
+    /**  */
+    public IgniteCheckedException error() {
+        return error;
+    }
+
+    /** */
+    public boolean successful() {
+        return resType == ResultType.SUCCESS;
+    }
+
+    /** */
+    public boolean exchangeDisabled() {
+        return resType == ResultType.EXCHANGE_DISABLED;
+    }
+
+    /**
+     * @param acceptedClsName Accepted class name.
+     */
+    static MappingExchangeResult createSuccessfulResult(String acceptedClsName) {
+        assert acceptedClsName != null;
+
+        return new MappingExchangeResult(ResultType.SUCCESS, acceptedClsName, null);
+    }
+
+    /**
+     * @param error Error.
+     */
+    static MappingExchangeResult createFailureResult(IgniteCheckedException error) {
+        assert error != null;
+
+        return new MappingExchangeResult(ResultType.FAILURE, null, error);
+    }
+
+    /** */
+    static MappingExchangeResult createExchangeDisabledResult() {
+        return new MappingExchangeResult(ResultType.EXCHANGE_DISABLED, null, null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
new file mode 100644
index 0000000..33a2168
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/marshaller/MappingProposedMessage.java
@@ -0,0 +1,137 @@
+/*
+ * 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.processors.marshaller;
+
+import java.util.UUID;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteUuid;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Node sends this message when it wants to propose new marshaller mapping and to ensure that there are no conflicts
+ * with this mapping on other nodes in cluster.
+ *
+ * After sending this message to the cluster sending node gets blocked until mapping is either accepted or rejected.
+ *
+ * When it completes a pass around the cluster ring with no conflicts observed,
+ * {@link MappingAcceptedMessage} is sent as an acknowledgement that everything is fine.
+ */
+public class MappingProposedMessage implements DiscoveryCustomMessage {
+    /** */
+    private enum ProposalStatus {
+        /** */
+        SUCCESSFUL,
+        /** */
+        IN_CONFLICT,
+        /** */
+        DUPLICATED
+    }
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
+    /** */
+    private final UUID origNodeId;
+
+    /** */
+    @GridToStringInclude
+    private final MarshallerMappingItem mappingItem;
+
+    /** */
+    private ProposalStatus status = ProposalStatus.SUCCESSFUL;
+
+    /** */
+    private String conflictingClsName;
+
+    /**
+     * @param mappingItem Mapping item.
+     * @param origNodeId Orig node id.
+     */
+    MappingProposedMessage(MarshallerMappingItem mappingItem, UUID origNodeId) {
+        assert origNodeId != null;
+
+        this.mappingItem = mappingItem;
+        this.origNodeId = origNodeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        if (status == ProposalStatus.SUCCESSFUL)
+            return new MappingAcceptedMessage(mappingItem);
+        else
+            return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return true;
+    }
+
+    /** */
+    MarshallerMappingItem mappingItem() {
+        return mappingItem;
+    }
+
+    /** */
+    UUID origNodeId() {
+        return origNodeId;
+    }
+
+    /** */
+    boolean inConflict() {
+        return status == ProposalStatus.IN_CONFLICT;
+    }
+
+    /** */
+    public boolean duplicated() {
+        return status == ProposalStatus.DUPLICATED;
+    }
+
+    /** */
+    void conflictingWithClass(String conflClsName) {
+        status = ProposalStatus.IN_CONFLICT;
+        conflictingClsName = conflClsName;
+    }
+
+    /** */
+    void markDuplicated() {
+        status = ProposalStatus.DUPLICATED;
+    }
+
+    /** */
+    String conflictingClassName() {
+        return conflictingClsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(MappingProposedMessage.class, this);
+    }
+}


[5/5] ignite git commit: IGNITE-4157 Use discovery custom messages instead of marshaller cache - Fixes #1271.

Posted by ag...@apache.org.
IGNITE-4157 Use  discovery custom messages instead of marshaller cache - Fixes #1271.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-2.0
Commit: 4cd332b781cf700b99402eed2363f988f6403602
Parents: 454b976
Author: Sergey Chugunov <se...@gmail.com>
Authored: Thu Jan 19 15:05:09 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 19 15:05:09 2017 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  72 --
 .../apache/ignite/internal/GridComponent.java   |  48 +-
 .../ignite/internal/GridKernalContext.java      |   7 -
 .../ignite/internal/GridKernalContextImpl.java  |  22 +-
 .../ignite/internal/GridPluginComponent.java    |  21 +-
 .../org/apache/ignite/internal/GridTopic.java   |   3 +
 .../apache/ignite/internal/IgniteKernal.java    |   6 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  50 --
 .../ignite/internal/MappingStoreTask.java       |  58 ++
 .../internal/MarshallerContextAdapter.java      | 211 ------
 .../ignite/internal/MarshallerContextImpl.java  | 664 +++++++++++++------
 .../internal/MarshallerMappingFileStore.java    | 174 +++++
 .../ignite/internal/MarshallerPlatformIds.java  |  30 +
 .../ignite/internal/binary/BinaryContext.java   |   8 +-
 .../GridClientOptimizedMarshaller.java          |  16 +-
 .../internal/managers/GridManagerAdapter.java   |  19 +-
 .../managers/communication/GridIoManager.java   |   2 -
 .../communication/GridIoMessageFactory.java     |  12 +
 .../managers/communication/GridIoPolicy.java    |   7 +-
 .../discovery/GridDiscoveryManager.java         |  83 ++-
 .../processors/GridProcessorAdapter.java        |  21 +-
 .../internal/processors/cache/CacheType.java    |   8 +-
 .../processors/cache/GridCacheAdapter.java      |   1 -
 .../processors/cache/GridCacheProcessor.java    | 275 ++++----
 .../processors/cache/GridCacheTtlManager.java   |   1 -
 .../processors/cache/GridCacheUtils.java        |  13 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |  13 +
 .../cache/query/GridCacheQueryManager.java      |   2 +-
 .../processors/cluster/ClusterProcessor.java    |  53 +-
 .../continuous/GridContinuousProcessor.java     | 144 ++--
 .../marshaller/ClientRequestFuture.java         | 183 +++++
 .../GridMarshallerMappingProcessor.java         | 326 +++++++++
 .../processors/marshaller/MappedName.java       |  63 ++
 .../marshaller/MappingAcceptedMessage.java      |  71 ++
 .../marshaller/MappingExchangeResult.java       |  96 +++
 .../marshaller/MappingProposedMessage.java      | 137 ++++
 .../marshaller/MarshallerMappingItem.java       |  99 +++
 .../marshaller/MarshallerMappingTransport.java  | 212 ++++++
 .../MissingMappingRequestMessage.java           | 146 ++++
 .../MissingMappingResponseMessage.java          | 169 +++++
 .../platform/utils/PlatformUtils.java           |  56 +-
 .../plugin/IgnitePluginProcessor.java           |  84 ++-
 .../internal/processors/pool/PoolProcessor.java |   5 -
 .../service/GridServiceProcessor.java           |   9 +-
 .../ignite/marshaller/MarshallerContext.java    |  23 +-
 .../optimized/OptimizedMarshallerUtils.java     |  16 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   0
 .../ignite/spi/discovery/DiscoveryDataBag.java  | 299 +++++++++
 .../spi/discovery/DiscoverySpiDataExchange.java |  15 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  26 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  98 +--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  71 +-
 .../tcp/internal/DiscoveryDataPacket.java       | 345 ++++++++++
 .../TcpDiscoveryJoinRequestMessage.java         |  18 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |  10 +-
 .../messages/TcpDiscoveryNodeAddedMessage.java  |  78 +--
 .../ignite/internal/GridAffinitySelfTest.java   |   2 +-
 .../MarshallerContextLockingSelfTest.java       |  61 +-
 .../binary/BinaryMarshallerSelfTest.java        |  20 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |  42 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   8 +-
 .../cache/IgniteInternalCacheTypesTest.java     |  11 -
 ...iteMarshallerCacheClassNameConflictTest.java | 273 ++++++++
 ...lerCacheClientRequestsMappingOnMissTest.java | 345 ++++++++++
 .../cache/IgniteSystemCacheOnClientTest.java    |  23 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |  21 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |  18 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   8 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   2 +-
 .../marshaller/MarshallerContextSelfTest.java   | 144 +++-
 .../marshaller/MarshallerContextTestImpl.java   |  34 +-
 .../OptimizedMarshallerEnumSelfTest.java        |  25 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   7 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 103 ++-
 .../tcp/TcpDiscoverySpiStartStopSelfTest.java   |  11 +-
 .../testframework/junits/GridAbstractTest.java  |  13 +-
 .../junits/GridTestKernalContext.java           |   1 -
 .../junits/spi/GridSpiAbstractTest.java         |   9 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |   5 +
 .../ignite/thread/IgniteThreadPoolSizeTest.java |   8 -
 81 files changed, 4597 insertions(+), 1298 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index e0ff9b9..512ceee 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -263,12 +263,6 @@ public class IgniteConfiguration {
     /** Utility cache pool keep alive time. */
     private long utilityCacheKeepAliveTime = DFLT_THREAD_KEEP_ALIVE_TIME;
 
-    /** Marshaller pool size. */
-    private int marshCachePoolSize = DFLT_SYSTEM_CORE_THREAD_CNT;
-
-    /** Marshaller pool keep alive time. */
-    private long marshCacheKeepAliveTime = DFLT_THREAD_KEEP_ALIVE_TIME;
-
     /** P2P pool size. */
     private int p2pPoolSize = DFLT_P2P_THREAD_CNT;
 
@@ -539,8 +533,6 @@ public class IgniteConfiguration {
         lsnrs = cfg.getLocalEventListeners();
         marsh = cfg.getMarshaller();
         marshLocJobs = cfg.isMarshalLocalJobs();
-        marshCacheKeepAliveTime = cfg.getMarshallerCacheKeepAliveTime();
-        marshCachePoolSize = cfg.getMarshallerCacheThreadPoolSize();
         mbeanSrv = cfg.getMBeanServer();
         metricsHistSize = cfg.getMetricsHistorySize();
         metricsExpTime = cfg.getMetricsExpireTime();
@@ -877,28 +869,6 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Default size of thread pool that is in charge of processing marshaller messages.
-     * <p>
-     * If not provided, executor service will have size {@link #DFLT_SYSTEM_CORE_THREAD_CNT}.
-     *
-     * @return Default thread pool size to be used in grid for marshaller messages.
-     */
-    public int getMarshallerCacheThreadPoolSize() {
-        return marshCachePoolSize;
-    }
-
-    /**
-     * Keep alive time of thread pool that is in charge of processing marshaller messages.
-     * <p>
-     * If not provided, executor service will have keep alive time {@link #DFLT_THREAD_KEEP_ALIVE_TIME}.
-     *
-     * @return Thread pool keep alive time (in milliseconds) to be used in grid for marshaller messages.
-     */
-    public long getMarshallerCacheKeepAliveTime() {
-        return marshCacheKeepAliveTime;
-    }
-
-    /**
      * Sets thread pool size to use within grid.
      *
      * @param poolSize Thread pool size to use within grid.
@@ -1019,48 +989,6 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Sets default thread pool size that will be used to process marshaller messages.
-     *
-     * @param poolSize Default executor service size to use for marshaller messages.
-     * @see IgniteConfiguration#getMarshallerCacheThreadPoolSize()
-     * @see IgniteConfiguration#getMarshallerCacheKeepAliveTime()
-     * @return {@code this} for chaining.
-     * @deprecated Use {@link #setMarshallerCacheThreadPoolSize(int)} instead.
-     */
-    @Deprecated
-    public IgniteConfiguration setMarshallerCachePoolSize(int poolSize) {
-        return setMarshallerCacheThreadPoolSize(poolSize);
-    }
-
-    /**
-     * Sets default thread pool size that will be used to process marshaller messages.
-     *
-     * @param poolSize Default executor service size to use for marshaller messages.
-     * @see IgniteConfiguration#getMarshallerCacheThreadPoolSize()
-     * @see IgniteConfiguration#getMarshallerCacheKeepAliveTime()
-     * @return {@code this} for chaining.
-     */
-    public IgniteConfiguration setMarshallerCacheThreadPoolSize(int poolSize) {
-        marshCachePoolSize = poolSize;
-
-        return this;
-    }
-
-    /**
-     * Sets maximum thread pool size that will be used to process marshaller messages.
-     *
-     * @param keepAliveTime Keep alive time of executor service to use for marshaller messages.
-     * @see IgniteConfiguration#getMarshallerCacheThreadPoolSize()
-     * @see IgniteConfiguration#getMarshallerCacheKeepAliveTime()
-     * @return {@code this} for chaining.
-     */
-    public IgniteConfiguration setMarshallerCacheKeepAliveTime(long keepAliveTime) {
-        marshCacheKeepAliveTime = keepAliveTime;
-
-        return this;
-    }
-
-    /**
      * Should return Ignite installation home folder. If not provided, the system will check
      * {@code IGNITE_HOME} system property and environment variable in that order. If
      * {@code IGNITE_HOME} still could not be obtained, then grid will not start and exception

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
index 5c77aee..560d7f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal;
 
-import java.io.Serializable;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -43,7 +46,10 @@ public interface GridComponent {
         PLUGIN,
 
         /** */
-        CLUSTER_PROC
+        CLUSTER_PROC,
+
+        /** */
+        MARSHALLER_PROC
     }
 
     /**
@@ -78,24 +84,37 @@ public interface GridComponent {
     public void onKernalStop(boolean cancel);
 
     /**
-     * Gets discovery data object that will be sent to new node
-     * during discovery process.
+     * Collects discovery data on joining node before sending
+     * {@link TcpDiscoveryJoinRequestMessage} request.
+     *
+     * @param dataBag container object to store discovery data in.
+     */
+    public void collectJoiningNodeData(DiscoveryDataBag dataBag);
+
+    /**
+     * Collects discovery data on nodes already in grid on receiving
+     * {@link TcpDiscoveryNodeAddedMessage}.
      *
-     * @param nodeId ID of new node that joins topology.
-     * @return Discovery data object or {@code null} if there is nothing
-     *      to send for this component.
+     * @param dataBag container object to store discovery data in.
      */
-    @Nullable public Serializable collectDiscoveryData(UUID nodeId);
+    public void collectGridNodeData(DiscoveryDataBag dataBag);
 
     /**
      * Receives discovery data object from remote nodes (called
      * on new node during discovery process).
      *
-     * @param joiningNodeId Joining node ID.
-     * @param rmtNodeId Remote node ID for which data is provided.
-     * @param data Discovery data object or {@code null} if nothing was
+     * @param data {@link GridDiscoveryData} interface to retrieve discovery data collected on remote nodes
+     *                                      (data common for all nodes in grid and specific for each node).
+     */
+    public void onGridDataReceived(GridDiscoveryData data);
+
+    /**
+     * Method is called on nodes that are already in grid (not on joining node).
+     * It receives discovery data from joining node.
+     *
+     * @param data {@link JoiningNodeDiscoveryData} interface to retrieve discovery data of joining node.
      */
-    public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data);
+    public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data);
 
     /**
      * Prints memory statistics (sizes of internal structures, etc.).
@@ -115,7 +134,8 @@ public interface GridComponent {
 
     /**
      * Gets unique component type to distinguish components providing discovery data. Must return non-null value
-     * if component implements method {@link #collectDiscoveryData(UUID)}.
+     * if component implements any of methods {@link #collectJoiningNodeData(DiscoveryDataBag)}
+     * or {@link #collectGridNodeData(DiscoveryDataBag)}.
      *
      * @return Unique component type for discovery data exchange.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 9157fed..b083c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -310,13 +310,6 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public ExecutorService utilityCachePool();
 
     /**
-     * Gets marshaller cache pool.
-     *
-     * @return Marshaller cache pool.
-     */
-    public ExecutorService marshallerCachePool();
-
-    /**
      * Gets async callback pool.
      *
      * @return Async callback pool.

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 8fc5b36..2f681ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -59,12 +59,13 @@ import org.apache.ignite.internal.processors.cluster.ClusterProcessor;
 import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
-import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
+import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.igfs.IgfsHelper;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
+import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.odbc.OdbcProcessor;
 import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor;
@@ -346,9 +347,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     private ExecutorService utilityCachePool;
 
     /** */
-    private ExecutorService marshCachePool;
-
-    /** */
     private IgniteConfiguration cfg;
 
     /** */
@@ -387,7 +385,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      * @param cfg Grid configuration.
      * @param gw Kernal gateway.
      * @param utilityCachePool Utility cache pool.
-     * @param marshCachePool Marshaller cache pool.
      * @param execSvc Public executor service.
      * @param sysExecSvc System executor service.
      * @param stripedExecSvc Striped executor.
@@ -408,7 +405,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         IgniteConfiguration cfg,
         GridKernalGateway gw,
         ExecutorService utilityCachePool,
-        ExecutorService marshCachePool,
         ExecutorService execSvc,
         ExecutorService sysExecSvc,
         StripedExecutor stripedExecSvc,
@@ -421,7 +417,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         @Nullable ExecutorService idxExecSvc,
         IgniteStripedThreadPoolExecutor callbackExecSvc,
         List<PluginProvider> plugins
-    ) throws IgniteCheckedException {
+    ) {
         assert grid != null;
         assert cfg != null;
         assert gw != null;
@@ -430,7 +426,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.cfg = cfg;
         this.gw = gw;
         this.utilityCachePool = utilityCachePool;
-        this.marshCachePool = marshCachePool;
         this.execSvc = execSvc;
         this.sysExecSvc = sysExecSvc;
         this.stripedExecSvc = stripedExecSvc;
@@ -443,9 +438,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.idxExecSvc = idxExecSvc;
         this.callbackExecSvc = callbackExecSvc;
 
-        String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome());
-
-        marshCtx = new MarshallerContextImpl(workDir, plugins);
+        marshCtx = new MarshallerContextImpl(plugins);
 
         try {
             spring = SPRING.create(false);
@@ -570,7 +563,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             platformProc = (PlatformProcessor)comp;
         else if (comp instanceof PoolProcessor)
             poolProc = (PoolProcessor) comp;
-        else if (!(comp instanceof DiscoveryNodeValidationProcessor))
+        else if (!(comp instanceof DiscoveryNodeValidationProcessor || comp instanceof GridMarshallerMappingProcessor))
             assert (comp instanceof GridPluginComponent) : "Unknown manager class: " + comp.getClass();
 
         if (addToList)
@@ -804,11 +797,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public ExecutorService marshallerCachePool() {
-        return marshCachePool;
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteStripedThreadPoolExecutor asyncCallbackPool() {
         return callbackExecSvc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
index 89dc243..cc1ae71 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
@@ -17,14 +17,15 @@
 
 package org.apache.ignite.internal;
 
-import java.io.Serializable;
-import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.plugin.PluginValidationException;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -85,12 +86,22 @@ public class GridPluginComponent implements GridComponent {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Serializable collectDiscoveryData(UUID nodeId) {
-        return null;
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 2962540..ce528cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -100,6 +100,9 @@ public enum GridTopic {
     TOPIC_IO_TEST,
 
     /** */
+    TOPIC_MAPPING_MARSH,
+
+    /** */
     TOPIC_HADOOP_MSG;
 
     /** Enum values. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/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 99c3dab..20926f9 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
@@ -118,6 +118,7 @@ import org.apache.ignite.internal.processors.hadoop.Hadoop;
 import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
+import org.apache.ignite.internal.processors.marshaller.GridMarshallerMappingProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.DiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.nodevalidation.OsDiscoveryNodeValidationProcessor;
 import org.apache.ignite.internal.processors.odbc.OdbcProcessor;
@@ -679,7 +680,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     public void start(
         final IgniteConfiguration cfg,
         ExecutorService utilityCachePool,
-        ExecutorService marshCachePool,
         final ExecutorService execSvc,
         final ExecutorService sysExecSvc,
         final StripedExecutor stripedExecSvc,
@@ -789,7 +789,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 cfg,
                 gw,
                 utilityCachePool,
-                marshCachePool,
                 execSvc,
                 sysExecSvc,
                 stripedExecSvc,
@@ -905,6 +904,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             startProcessor(createHadoopComponent());
             startProcessor(new DataStructuresProcessor(ctx));
             startProcessor(createComponent(PlatformProcessor.class, ctx));
+            startProcessor(new GridMarshallerMappingProcessor(ctx));
 
             // Start plugins.
             for (PluginProvider provider : ctx.plugins().allProviders()) {
@@ -1967,8 +1967,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             List<GridComponent> comps = ctx.components();
 
-            ctx.marshallerContext().onKernalStop();
-
             // Callback component in reverse order while kernal is still functional
             // if called in the same thread, at least.
             for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 9fe6fd0..c55f954 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -114,7 +114,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_LOCAL_HOST;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_SHUTDOWN_HOOK;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_RESTART_CODE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -1481,9 +1480,6 @@ public class IgnitionEx {
         /** Utility cache executor service. */
         private ThreadPoolExecutor utilityCacheExecSvc;
 
-        /** Marshaller cache executor service. */
-        private ThreadPoolExecutor marshCacheExecSvc;
-
         /** Affinity executor service. */
         private ThreadPoolExecutor affExecSvc;
 
@@ -1764,18 +1760,6 @@ public class IgnitionEx {
 
             utilityCacheExecSvc.allowCoreThreadTimeOut(true);
 
-            validateThreadPoolSize(myCfg.getMarshallerCacheThreadPoolSize(), "marshaller cache");
-
-            marshCacheExecSvc = new IgniteThreadPoolExecutor(
-                "marshaller-cache",
-                cfg.getGridName(),
-                myCfg.getMarshallerCacheThreadPoolSize(),
-                myCfg.getMarshallerCacheThreadPoolSize(),
-                myCfg.getMarshallerCacheKeepAliveTime(),
-                new LinkedBlockingQueue<Runnable>());
-
-            marshCacheExecSvc.allowCoreThreadTimeOut(true);
-
             affExecSvc = new IgniteThreadPoolExecutor(
                 "aff",
                 cfg.getGridName(),
@@ -1813,7 +1797,6 @@ public class IgnitionEx {
                 grid0.start(
                     myCfg,
                     utilityCacheExecSvc,
-                    marshCacheExecSvc,
                     execSvc,
                     sysExecSvc,
                     stripedExecSvc,
@@ -2068,8 +2051,6 @@ public class IgnitionEx {
         public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
             List<CacheConfiguration> cacheCfgs = new ArrayList<>();
 
-            cacheCfgs.add(marshallerSystemCache());
-
             cacheCfgs.add(utilitySystemCache());
 
             if (IgniteComponentType.HADOOP.inClassPath())
@@ -2098,10 +2079,6 @@ public class IgnitionEx {
                         throw new IgniteCheckedException("Cache name cannot be \"" + CU.UTILITY_CACHE_NAME +
                             "\" because it is reserved for internal purposes.");
 
-                    if (CU.isMarshallerCache(ccfg.getName()))
-                        throw new IgniteCheckedException("Cache name cannot be \"" + CU.MARSH_CACHE_NAME +
-                            "\" because it is reserved for internal purposes.");
-
                     cacheCfgs.add(ccfg);
                 }
             }
@@ -2277,29 +2254,6 @@ public class IgnitionEx {
         }
 
         /**
-         * Creates marshaller system cache configuration.
-         *
-         * @return Marshaller system cache configuration.
-         */
-        private static CacheConfiguration marshallerSystemCache() {
-            CacheConfiguration cache = new CacheConfiguration();
-
-            cache.setName(CU.MARSH_CACHE_NAME);
-            cache.setCacheMode(REPLICATED);
-            cache.setAtomicityMode(ATOMIC);
-            cache.setSwapEnabled(false);
-            cache.setRebalanceMode(SYNC);
-            cache.setWriteSynchronizationMode(FULL_SYNC);
-            cache.setAffinity(new RendezvousAffinityFunction(false, 20));
-            cache.setNodeFilter(CacheConfiguration.ALL_NODES);
-            cache.setStartSize(300);
-            cache.setRebalanceOrder(-2);//Prior to other system caches.
-            cache.setCopyOnRead(false);
-
-            return cache;
-        }
-
-        /**
          * Creates utility system cache configuration.
          *
          * @return Utility system cache configuration.
@@ -2477,10 +2431,6 @@ public class IgnitionEx {
 
             utilityCacheExecSvc = null;
 
-            U.shutdownNow(getClass(), marshCacheExecSvc, log);
-
-            marshCacheExecSvc = null;
-
             U.shutdownNow(getClass(), affExecSvc, log);
 
             affExecSvc = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/MappingStoreTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MappingStoreTask.java b/modules/core/src/main/java/org/apache/ignite/internal/MappingStoreTask.java
new file mode 100644
index 0000000..102347a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MappingStoreTask.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;
+
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem;
+
+/**
+ * Task is used in {@link MarshallerContextImpl#onMappingAccepted(MarshallerMappingItem)}
+ * to offload storing mapping data into file system from discovery thread.
+ */
+class MappingStoreTask implements Runnable {
+    /** Store to put item to. */
+    private final MarshallerMappingFileStore fileStore;
+
+    /** */
+    private final byte platformId;
+
+    /** */
+    private final int typeId;
+
+    /** */
+    private final String clsName;
+
+    /**
+     * @param fileStore File store.
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     * @param clsName Class name.
+     */
+    MappingStoreTask(MarshallerMappingFileStore fileStore, byte platformId, int typeId, String clsName) {
+        assert clsName != null;
+
+        this.fileStore = fileStore;
+        this.platformId = platformId;
+        this.typeId = typeId;
+        this.clsName = clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run() {
+        fileStore.writeMapping(platformId, typeId, clsName);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
deleted file mode 100644
index ad34393..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
+++ /dev/null
@@ -1,211 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.net.URL;
-import java.util.Enumeration;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.MarshallerContext;
-import org.apache.ignite.plugin.PluginProvider;
-import org.jetbrains.annotations.Nullable;
-import org.jsr166.ConcurrentHashMap8;
-
-/**
- * Marshaller context adapter.
- */
-public abstract class MarshallerContextAdapter implements MarshallerContext {
-    /** */
-    private static final String CLS_NAMES_FILE = "META-INF/classnames.properties";
-
-    /** */
-    private static final String JDK_CLS_NAMES_FILE = "META-INF/classnames-jdk.properties";
-
-    /** */
-    private final ConcurrentMap<Integer, String> map = new ConcurrentHashMap8<>();
-
-    /** */
-    private final Set<String> registeredSystemTypes = new HashSet<>();
-
-    /**
-     * Initializes context.
-     *
-     * @param plugins Plugins.
-     */
-    public MarshallerContextAdapter(@Nullable List<PluginProvider> plugins) {
-        try {
-            ClassLoader ldr = U.gridClassLoader();
-
-            Enumeration<URL> urls = ldr.getResources(CLS_NAMES_FILE);
-
-            boolean foundClsNames = false;
-
-            while (urls.hasMoreElements()) {
-                processResource(urls.nextElement());
-
-                foundClsNames = true;
-            }
-
-            if (!foundClsNames)
-                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
-                    "[file=" + CLS_NAMES_FILE + ", ldr=" + ldr + ']');
-
-            URL jdkClsNames = ldr.getResource(JDK_CLS_NAMES_FILE);
-
-            if (jdkClsNames == null)
-                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
-                    "[file=" + JDK_CLS_NAMES_FILE + ", ldr=" + ldr + ']');
-
-            processResource(jdkClsNames);
-
-            checkHasClassName(GridDhtPartitionFullMap.class.getName(), ldr, CLS_NAMES_FILE);
-            checkHasClassName(GridDhtPartitionMap2.class.getName(), ldr, CLS_NAMES_FILE);
-            checkHasClassName(HashMap.class.getName(), ldr, JDK_CLS_NAMES_FILE);
-
-            if (plugins != null && !plugins.isEmpty()) {
-                for (PluginProvider plugin : plugins) {
-                    URL pluginClsNames = ldr.getResource("META-INF/" + plugin.name().toLowerCase()
-                        + ".classnames.properties");
-
-                    if (pluginClsNames != null)
-                        processResource(pluginClsNames);
-                }
-            }
-        }
-        catch (IOException e) {
-            throw new IllegalStateException("Failed to initialize marshaller context.", e);
-        }
-    }
-
-    /**
-     * @param clsName Class name.
-     * @param ldr Class loader used to get properties file.
-     * @param fileName File name.
-     */
-    public void checkHasClassName(String clsName, ClassLoader ldr, String fileName) {
-        if (!map.containsKey(clsName.hashCode()))
-            throw new IgniteException("Failed to read class name from class names properties file. " +
-                "Make sure class names properties file packaged with ignite binaries is not corrupted " +
-                "[clsName=" + clsName + ", fileName=" + fileName + ", ldr=" + ldr + ']');
-    }
-
-    /**
-     * @param url Resource URL.
-     * @throws IOException In case of error.
-     */
-    private void processResource(URL url) throws IOException {
-        try (InputStream in = url.openStream()) {
-            BufferedReader rdr = new BufferedReader(new InputStreamReader(in));
-
-            String line;
-
-            while ((line = rdr.readLine()) != null) {
-                if (line.isEmpty() || line.startsWith("#"))
-                    continue;
-
-                String clsName = line.trim();
-
-                int typeId = clsName.hashCode();
-
-                String oldClsName;
-
-                if ((oldClsName = map.put(typeId, clsName)) != null) {
-                    if (!oldClsName.equals(clsName))
-                        throw new IgniteException("Duplicate type ID [id=" + typeId + ", clsName=" + clsName +
-                        ", oldClsName=" + oldClsName + ']');
-                }
-
-                registeredSystemTypes.add(clsName);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean registerClass(int id, Class cls) throws IgniteCheckedException {
-        boolean registered = true;
-
-        String clsName = map.get(id);
-
-        if (clsName == null) {
-            registered = registerClassName(id, cls.getName());
-
-            if (registered)
-                map.putIfAbsent(id, cls.getName());
-        }
-        else if (!clsName.equals(cls.getName()))
-            throw new IgniteCheckedException("Duplicate ID [id=" + id + ", oldCls=" + clsName +
-                ", newCls=" + cls.getName());
-
-        return registered;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Class getClass(int id, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException {
-        String clsName = map.get(id);
-
-        if (clsName == null) {
-            clsName = className(id);
-
-            if (clsName == null)
-                throw new ClassNotFoundException("Unknown type ID: " + id);
-
-            String old = map.putIfAbsent(id, clsName);
-
-            if (old != null)
-                clsName = old;
-        }
-
-        return U.forName(clsName, ldr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isSystemType(String typeName) {
-        return registeredSystemTypes.contains(typeName);
-    }
-
-    /**
-     * Registers class name.
-     *
-     * @param id Type ID.
-     * @param clsName Class name.
-     * @return Whether class name was registered.
-     * @throws IgniteCheckedException In case of error.
-     */
-    protected abstract boolean registerClassName(int id, String clsName) throws IgniteCheckedException;
-
-    /**
-     * Gets class name by type ID.
-     *
-     * @param id Type ID.
-     * @return Class name.
-     * @throws IgniteCheckedException In case of error.
-     */
-    protected abstract String className(int id) throws IgniteCheckedException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index f3e368d..751a7d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -18,310 +18,564 @@
 package org.apache.ignite.internal;
 
 import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
-import java.nio.charset.StandardCharsets;
+import java.net.URL;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Enumeration;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.locks.Lock;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.CacheEntryListenerException;
-import javax.cache.event.CacheEntryUpdatedListener;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.processors.cache.CachePartialUpdateCheckedException;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheTryPutFailedException;
-import org.apache.ignite.internal.util.GridStripedLock;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.marshaller.MappedName;
+import org.apache.ignite.internal.processors.marshaller.MappingExchangeResult;
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem;
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingTransport;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.plugin.PluginProvider;
+import org.jetbrains.annotations.NotNull;
+import org.jetbrains.annotations.Nullable;
+import org.jsr166.ConcurrentHashMap8;
+
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
 
 /**
  * Marshaller context implementation.
  */
-public class MarshallerContextImpl extends MarshallerContextAdapter {
+public class MarshallerContextImpl implements MarshallerContext {
+    /** */
+    private static final String CLS_NAMES_FILE = "META-INF/classnames.properties";
+
     /** */
-    private static final GridStripedLock fileLock = new GridStripedLock(32);
+    private static final String JDK_CLS_NAMES_FILE = "META-INF/classnames-jdk.properties";
 
     /** */
-    private final CountDownLatch latch = new CountDownLatch(1);
+    private final Map<Integer, MappedName> sysTypesMap = new HashMap<>();
 
     /** */
-    private final File workDir;
+    private final Collection<String> sysTypesSet = new HashSet<>();
 
     /** */
-    private IgniteLogger log;
+    private final List<ConcurrentMap<Integer, MappedName>> allCaches = new CopyOnWriteArrayList<>();
 
     /** */
-    private volatile GridCacheAdapter<Integer, String> cache;
+    private MarshallerMappingFileStore fileStore;
 
-    /** Non-volatile on purpose. */
-    private int failedCnt;
+    /** */
+    private ExecutorService execSrvc;
+
+    /** */
+    private MarshallerMappingTransport transport;
 
     /** */
-    private ContinuousQueryListener lsnr;
+    private boolean isClientNode;
 
     /**
-     * @param igniteWorkDir Ignite work directory.
+     * Initializes context.
+     *
      * @param plugins Plugins.
-     * @throws IgniteCheckedException In case of error.
      */
-    public MarshallerContextImpl(String igniteWorkDir, List<PluginProvider> plugins) throws IgniteCheckedException {
-        super(plugins);
+    public MarshallerContextImpl(@Nullable Collection<PluginProvider> plugins) {
+        initializeCaches();
+
+        try {
+            ClassLoader ldr = U.gridClassLoader();
+
+            Enumeration<URL> urls = ldr.getResources(CLS_NAMES_FILE);
+
+            boolean foundClsNames = false;
+
+            while (urls.hasMoreElements()) {
+                processResource(urls.nextElement());
 
-        workDir = U.resolveWorkDirectory(igniteWorkDir, "marshaller", false);
+                foundClsNames = true;
+            }
+
+            if (!foundClsNames)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            URL jdkClsNames = ldr.getResource(JDK_CLS_NAMES_FILE);
+
+            if (jdkClsNames == null)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + JDK_CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            processResource(jdkClsNames);
+
+            checkHasClassName(GridDhtPartitionFullMap.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(GridDhtPartitionMap2.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(HashMap.class.getName(), ldr, JDK_CLS_NAMES_FILE);
+
+            if (plugins != null && !plugins.isEmpty()) {
+                for (PluginProvider plugin : plugins) {
+                    URL pluginClsNames = ldr.getResource("META-INF/" + plugin.name().toLowerCase()
+                        + ".classnames.properties");
+
+                    if (pluginClsNames != null)
+                        processResource(pluginClsNames);
+                }
+            }
+        }
+        catch (IOException e) {
+            throw new IllegalStateException("Failed to initialize marshaller context.", e);
+        }
+    }
+
+    /** */
+    private void initializeCaches() {
+        allCaches.add(new CombinedMap(new ConcurrentHashMap8<Integer, MappedName>(), sysTypesMap));
+    }
+
+    /** */
+    public ArrayList<Map<Integer, MappedName>> getCachedMappings() {
+        ArrayList<Map<Integer, MappedName>> result = new ArrayList<>(allCaches.size());
+
+        for (int i = 0; i < allCaches.size(); i++) {
+            Map res;
+
+            if (i == JAVA_ID)
+                res = ((CombinedMap) allCaches.get(JAVA_ID)).userMap;
+            else
+                res = allCaches.get(i);
+
+            if (!res.isEmpty())
+                result.add(res);
+        }
+
+        return result;
     }
 
     /**
-     * @param ctx Context.
-     * @throws IgniteCheckedException If failed.
+     * @param platformId Platform id.
+     * @param marshallerMapping Marshaller mapping.
      */
-    public void onContinuousProcessorStarted(GridKernalContext ctx) throws IgniteCheckedException {
-        if (ctx.clientNode()) {
-            lsnr = new ContinuousQueryListener(ctx.log(MarshallerContextImpl.class), workDir);
-
-            ctx.continuous().registerStaticRoutine(
-                CU.MARSH_CACHE_NAME,
-                lsnr,
-                null,
-                null);
-        }
+    public void onMappingDataReceived(byte platformId, Map<Integer, MappedName> marshallerMapping) {
+        ConcurrentMap<Integer, MappedName> platformCache = getCacheFor(platformId);
+
+        for (Map.Entry<Integer, MappedName> e : marshallerMapping.entrySet())
+            platformCache.put(e.getKey(), new MappedName(e.getValue().className(), true));
     }
 
     /**
-     * @param ctx Kernal context.
-     * @throws IgniteCheckedException In case of error.
+     * @param clsName Class name.
+     * @param ldr Class loader used to get properties file.
+     * @param fileName File name.
      */
-    public void onMarshallerCacheStarted(GridKernalContext ctx) throws IgniteCheckedException {
-        assert ctx != null;
+    private void checkHasClassName(String clsName, ClassLoader ldr, String fileName) {
+        ConcurrentMap cache = getCacheFor(JAVA_ID);
+
+        if (!cache.containsKey(clsName.hashCode()))
+            throw new IgniteException("Failed to read class name from class names properties file. " +
+                "Make sure class names properties file packaged with ignite binaries is not corrupted " +
+                "[clsName=" + clsName + ", fileName=" + fileName + ", ldr=" + ldr + ']');
+    }
+
+    /**
+     * @param url Resource URL.
+     * @throws IOException In case of error.
+     */
+    private void processResource(URL url) throws IOException {
+        try (InputStream in = url.openStream()) {
+            BufferedReader rdr = new BufferedReader(new InputStreamReader(in));
+
+            String line;
+
+            while ((line = rdr.readLine()) != null) {
+                if (line.isEmpty() || line.startsWith("#"))
+                    continue;
 
-        log = ctx.log(MarshallerContextImpl.class);
+                String clsName = line.trim();
 
-        cache = ctx.cache().marshallerCache();
+                int typeId = clsName.hashCode();
 
-        if (ctx.cache().marshallerCache().context().affinityNode()) {
-            ctx.cache().marshallerCache().context().continuousQueries().executeInternalQuery(
-                new ContinuousQueryListener(log, workDir),
-                null,
-                true,
-                true,
-                false
-            );
+                MappedName oldClsName;
+
+                if ((oldClsName = sysTypesMap.put(typeId, new MappedName(clsName, true))) != null) {
+                    if (!oldClsName.className().equals(clsName))
+                        throw new IgniteException(
+                                "Duplicate type ID [id="
+                                        + typeId
+                                        + ", oldClsName="
+                                        + oldClsName
+                                        + ", clsName="
+                                        + clsName + ']');
+                }
+
+                sysTypesSet.add(clsName);
+            }
         }
-        else {
-            if (lsnr != null) {
-                ctx.closure().runLocalSafe(new Runnable() {
-                    @SuppressWarnings("unchecked")
-                    @Override public void run() {
-                        try {
-                            Iterable entries = cache.context().continuousQueries().existingEntries(false, null);
-
-                            lsnr.onUpdated(entries);
-                        }
-                        catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to load marshaller cache entries: " + e, e);
-                        }
-                    }
-                });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean registerClassName(
+            byte platformId,
+            int typeId,
+            String clsName
+    ) throws IgniteCheckedException {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(platformId);
+
+        MappedName mappedName = cache.get(typeId);
+
+        if (mappedName != null) {
+            if (!mappedName.className().equals(clsName))
+                throw duplicateIdException(platformId, typeId, mappedName.className(), clsName);
+            else {
+                if (mappedName.accepted())
+                    return true;
+
+                if (transport.stopping())
+                    return false;
+
+                IgniteInternalFuture<MappingExchangeResult> fut = transport.awaitMappingAcceptance(new MarshallerMappingItem(platformId, typeId, clsName), cache);
+                MappingExchangeResult res = fut.get();
+
+                return convertXchRes(res);
             }
         }
+        else {
+            if (transport.stopping())
+                return false;
 
-        latch.countDown();
+            IgniteInternalFuture<MappingExchangeResult> fut = transport.proposeMapping(new MarshallerMappingItem(platformId, typeId, clsName), cache);
+            MappingExchangeResult res = fut.get();
+
+            return convertXchRes(res);
+        }
+    }
+
+    /**
+     * @param res result of exchange.
+     */
+    private boolean convertXchRes(MappingExchangeResult res) throws IgniteCheckedException {
+        if (res.successful())
+            return true;
+        else if (res.exchangeDisabled())
+            return false;
+        else {
+            assert res.error() != null;
+            throw res.error();
+        }
     }
 
     /**
-     * Release marshaller context.
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     * @param conflictingClsName Conflicting class name.
+     * @param clsName Class name.
      */
-    public void onKernalStop() {
-        latch.countDown();
+    private IgniteCheckedException duplicateIdException(
+            byte platformId,
+            int typeId,
+            String conflictingClsName,
+            String clsName
+    ) {
+        return new IgniteCheckedException("Duplicate ID [platformId="
+                + platformId
+                + ", typeId="
+                + typeId
+                + ", oldCls="
+                + conflictingClsName
+                + ", newCls="
+                + clsName + "]");
+    }
+
+    /**
+     *
+     * @param item type mapping to propose
+     * @return false if there is a conflict with another mapping in local cache, true otherwise.
+     */
+    public String onMappingProposed(MarshallerMappingItem item) {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(item.platformId());
+
+        MappedName newName = new MappedName(item.className(), false);
+        MappedName oldName;
+
+        if ((oldName = cache.putIfAbsent(item.typeId(), newName)) == null)
+            return null;
+        else
+            return oldName.className();
+    }
+
+    /**
+     * @param item Item.
+     */
+    public void onMappingAccepted(final MarshallerMappingItem item) {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(item.platformId());
+
+        cache.replace(item.typeId(), new MappedName(item.className(), true));
+
+        execSrvc.submit(new MappingStoreTask(fileStore, item.platformId(), item.typeId(), item.className()));
     }
 
     /** {@inheritDoc} */
-    @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
-        GridCacheAdapter<Integer, String> cache0 = cache;
+    @Override public Class getClass(int typeId, ClassLoader ldr) throws ClassNotFoundException, IgniteCheckedException {
+        String clsName = getClassName(JAVA_ID, typeId);
 
-        if (cache0 == null)
-            return false;
+        if (clsName == null)
+            throw new ClassNotFoundException("Unknown type ID: " + typeId);
 
-        String old;
+        return U.forName(clsName, ldr);
+    }
 
-        try {
-            old = cache0.tryGetAndPut(id, clsName);
+    /** {@inheritDoc} */
+    @Override public String getClassName(
+            byte platformId,
+            int typeId
+    ) throws ClassNotFoundException, IgniteCheckedException {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(platformId);
 
-            if (old != null && !old.equals(clsName))
-                throw new IgniteCheckedException("Type ID collision detected [id=" + id + ", clsName1=" + clsName +
-                    ", clsName2=" + old + ']');
+        MappedName mappedName = cache.get(typeId);
 
-            failedCnt = 0;
+        String clsName;
 
-            return true;
-        }
-        catch (CachePartialUpdateCheckedException | GridCacheTryPutFailedException e) {
-            if (++failedCnt > 10) {
-                if (log.isQuiet())
-                    U.quiet(false, "Failed to register marshalled class for more than 10 times in a row " +
-                        "(may affect performance).");
+        if (mappedName != null)
+            clsName = mappedName.className();
+        else {
+            clsName = fileStore.readMapping(platformId, typeId);
 
-                failedCnt = 0;
-            }
+            if (clsName != null)
+                cache.putIfAbsent(typeId, new MappedName(clsName, true));
+            else
+                if (isClientNode) {
+                    mappedName = cache.get(typeId);
 
-            return false;
+                    if (mappedName == null) {
+                        GridFutureAdapter<MappingExchangeResult> fut = transport.requestMapping(
+                                new MarshallerMappingItem(platformId, typeId, null),
+                                cache);
+
+                        clsName = fut.get().className();
+                    }
+                    else
+                        clsName = mappedName.className();
+
+                    if (clsName == null)
+                        throw new ClassNotFoundException(
+                                "Requesting mapping from grid failed for [platformId="
+                                        + platformId
+                                        + ", typeId="
+                                        + typeId + "]");
+
+                    return clsName;
+                }
+                else
+                    throw new ClassNotFoundException(
+                            "Unknown pair [platformId= "
+                                    + platformId
+                                    + ", typeId="
+                                    + typeId + "]");
         }
+
+        return clsName;
     }
 
-    /** {@inheritDoc} */
-    @Override public String className(int id) throws IgniteCheckedException {
-        GridCacheAdapter<Integer, String> cache0 = cache;
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     */
+    public String resolveMissedMapping(byte platformId, int typeId) {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(platformId);
 
-        if (cache0 == null) {
-            U.awaitQuiet(latch);
+        MappedName mappedName = cache.get(typeId);
 
-            cache0 = cache;
+        if (mappedName != null) {
+            assert mappedName.accepted() : mappedName;
 
-            if (cache0 == null)
-                throw new IllegalStateException("Failed to initialize marshaller context (grid is stopping).");
+            return mappedName.className();
         }
 
-        String clsName = cache0.getTopologySafe(id);
+        return null;
+    }
+
+    /**
+     * @param item Item.
+     * @param resolvedClsName Resolved class name.
+     */
+    public void onMissedMappingResolved(MarshallerMappingItem item, String resolvedClsName) {
+        ConcurrentMap<Integer, MappedName> cache = getCacheFor(item.platformId());
+
+        int typeId = item.typeId();
+        MappedName mappedName = cache.get(typeId);
+
+        if (mappedName != null)
+            assert resolvedClsName.equals(mappedName.className()) :
+                    "Class name resolved from cluster: "
+                            + resolvedClsName
+                            + ", class name from local cache: "
+                            + mappedName.className();
+        else {
+            mappedName = new MappedName(resolvedClsName, true);
+            cache.putIfAbsent(typeId, mappedName);
+
+            execSrvc.submit(new MappingStoreTask(fileStore, item.platformId(), item.typeId(), resolvedClsName));
+        }
+    }
 
-        if (clsName == null) {
-            String fileName = id + ".classname";
+    /** {@inheritDoc} */
+    @Override public boolean isSystemType(String typeName) {
+        return sysTypesSet.contains(typeName);
+    }
 
-            Lock lock = fileLock(fileName);
+    /**
+     * @param platformId Platform id.
+     */
+    private ConcurrentMap<Integer, MappedName> getCacheFor(byte platformId) {
+        ConcurrentMap<Integer, MappedName> map;
 
-            lock.lock();
+        if (platformId < allCaches.size()) {
+            map = allCaches.get(platformId);
 
-            try {
-                File file = new File(workDir, fileName);
+            if (map != null)
+                return map;
+        }
 
-                try (FileInputStream in = new FileInputStream(file)) {
-                    FileLock fileLock = fileLock(in.getChannel(), true);
+        synchronized (this) {
+            int size = allCaches.size();
 
-                    assert fileLock != null : fileName;
+            if (platformId < size) {
+                map = allCaches.get(platformId);
 
-                    try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) {
-                        clsName = reader.readLine();
-                    }
+                if (map == null) {
+                    map = new ConcurrentHashMap8<>();
+                    allCaches.set(platformId, map);
                 }
-                catch (IOException e) {
-                    throw new IgniteCheckedException("Class definition was not found " +
-                        "at marshaller cache and local file. " +
-                        "[id=" + id + ", file=" + file.getAbsolutePath() + ']');
-                }
-            }
-            finally {
-                lock.unlock();
             }
+            else {
+                map = new ConcurrentHashMap8<>();
 
-            // Must explicitly put entry to cache to invoke other continuous queries.
-            registerClassName(id, clsName);
+                putAtIndex(map, allCaches, platformId, size);
+            }
         }
 
-        return clsName;
+        return map;
     }
 
     /**
-     * @param fileName File name.
-     * @return Lock instance.
+     * @param map Map.
+     * @param allCaches All caches.
+     * @param targetIdx Target index.
+     * @param size Size.
      */
-    private static Lock fileLock(String fileName) {
-        return fileLock.getLock(fileName.hashCode());
+    private static void putAtIndex(
+            ConcurrentMap<Integer, MappedName> map,
+            Collection<ConcurrentMap<Integer, MappedName>> allCaches,
+            byte targetIdx,
+            int size
+    ) {
+        int lastIdx = size - 1;
+
+        int nullElemsToAdd = targetIdx - lastIdx - 1;
+
+        for (int i = 0; i < nullElemsToAdd; i++)
+            allCaches.add(null);
+
+        allCaches.add(map);
     }
 
     /**
-     * @param ch File channel.
-     * @param shared Shared.
+     * @param ctx Context.
+     * @param transport Transport.
      */
-    private static FileLock fileLock(
-        FileChannel ch,
-        boolean shared
-    ) throws IOException, IgniteInterruptedCheckedException {
-        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+    public void onMarshallerProcessorStarted(
+            GridKernalContext ctx,
+            MarshallerMappingTransport transport
+    ) throws IgniteCheckedException {
+        assert ctx != null;
 
-        while (true) {
-            FileLock fileLock = ch.tryLock(0L, Long.MAX_VALUE, shared);
+        IgniteConfiguration cfg = ctx.config();
+        String workDir = U.workDirectory(cfg.getWorkDirectory(), cfg.getIgniteHome());
 
-            if (fileLock == null)
-                U.sleep(rnd.nextLong(50));
-            else
-                return fileLock;
-        }
+        fileStore = new MarshallerMappingFileStore(workDir, ctx.log(MarshallerMappingFileStore.class));
+        this.transport = transport;
+        execSrvc = ctx.getSystemExecutorService();
+        isClientNode = ctx.clientNode();
+    }
+
+    /**
+     *
+     */
+    public void onMarshallerProcessorStop() {
+        transport.markStopping();
     }
 
     /**
+     *
      */
-    public static class ContinuousQueryListener implements CacheEntryUpdatedListener<Integer, String> {
+    static final class CombinedMap extends AbstractMap<Integer, MappedName>
+            implements ConcurrentMap<Integer, MappedName> {
         /** */
-        private final IgniteLogger log;
+        private final ConcurrentMap<Integer, MappedName> userMap;
 
         /** */
-        private final File workDir;
+        private final Map<Integer, MappedName> sysMap;
 
         /**
-         * @param log Logger.
-         * @param workDir Work directory.
+         * @param userMap User map.
+         * @param sysMap System map.
          */
-        public ContinuousQueryListener(IgniteLogger log, File workDir) {
-            this.log = log;
-            this.workDir = workDir;
+        CombinedMap(ConcurrentMap<Integer, MappedName> userMap, Map<Integer, MappedName> sysMap) {
+            this.userMap = userMap;
+            this.sysMap = sysMap;
         }
 
         /** {@inheritDoc} */
-        @Override public void onUpdated(Iterable<CacheEntryEvent<? extends Integer, ? extends String>> evts)
-            throws CacheEntryListenerException {
-            for (CacheEntryEvent<? extends Integer, ? extends String> evt : evts) {
-                assert evt.getOldValue() == null || F.eq(evt.getOldValue(), evt.getValue()):
-                    "Received cache entry update for system marshaller cache: " + evt;
-
-                if (evt.getOldValue() == null) {
-                    String fileName = evt.getKey() + ".classname";
-
-                    Lock lock = fileLock(fileName);
-
-                    lock.lock();
-
-                    try {
-                        File file = new File(workDir, fileName);
-
-                        try (FileOutputStream out = new FileOutputStream(file)) {
-                            FileLock fileLock = fileLock(out.getChannel(), false);
-
-                            assert fileLock != null : fileName;
-
-                            try (Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
-                                writer.write(evt.getValue());
-
-                                writer.flush();
-                            }
-                        }
-                        catch (IOException e) {
-                            U.error(log, "Failed to write class name to file [id=" + evt.getKey() +
-                                ", clsName=" + evt.getValue() + ", file=" + file.getAbsolutePath() + ']', e);
-                        }
-                        catch(OverlappingFileLockException ignored) {
-                            if (log.isDebugEnabled())
-                                log.debug("File already locked (will ignore): " + file.getAbsolutePath());
-                        }
-                        catch (IgniteInterruptedCheckedException e) {
-                            U.error(log, "Interrupted while waiting for acquiring file lock: " + file, e);
-                        }
-                    }
-                    finally {
-                        lock.unlock();
-                    }
-                }
-            }
+        @Override public Set<Entry<Integer, MappedName>> entrySet() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public MappedName putIfAbsent(@NotNull Integer key, MappedName val) {
+            return userMap.putIfAbsent(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean remove(@NotNull Object key, Object val) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean replace(@NotNull Integer key, @NotNull MappedName oldVal, @NotNull MappedName newVal) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public MappedName replace(@NotNull Integer key, @NotNull MappedName val) {
+            return userMap.replace(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public MappedName get(Object key) {
+            MappedName res = sysMap.get(key);
+
+            if (res != null)
+                return res;
+
+            return userMap.get(key);
+        }
+
+        /** {@inheritDoc} */
+        @Override public MappedName put(Integer key, MappedName val) {
+            return userMap.put(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean containsKey(Object key) {
+            return userMap.containsKey(key) || sysMap.containsKey(key);
         }
     }
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java
new file mode 100644
index 0000000..03f79c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerMappingFileStore.java
@@ -0,0 +1,174 @@
+/*
+ * 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;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.Writer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+import java.nio.charset.StandardCharsets;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.locks.Lock;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.GridStripedLock;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * File-based persistence provider for {@link MarshallerContextImpl}.
+ *
+ * Saves mappings in format <b>{typeId}.classname{platformId}</b>, e.g. 123.classname0.
+ *
+ * It writes new mapping when it is accepted by all grid members and reads mapping
+ * when a classname is requested but is not presented in local cache of {@link MarshallerContextImpl}.
+ */
+final class MarshallerMappingFileStore {
+    /** */
+    private static final GridStripedLock fileLock = new GridStripedLock(32);
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final File workDir;
+
+    /**
+     * @param log Logger.
+     */
+    MarshallerMappingFileStore(String igniteWorkDir, IgniteLogger log) throws IgniteCheckedException {
+        workDir = U.resolveWorkDirectory(igniteWorkDir, "marshaller", false);
+        this.log = log;
+    }
+
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     * @param typeName Type name.
+     */
+    void writeMapping(byte platformId, int typeId, String typeName) {
+        String fileName = getFileName(platformId, typeId);
+
+        Lock lock = fileLock(fileName);
+
+        lock.lock();
+
+        try {
+            File file = new File(workDir, fileName);
+
+            try (FileOutputStream out = new FileOutputStream(file)) {
+                FileLock fileLock = fileLock(out.getChannel(), false);
+
+                assert fileLock != null : fileName;
+
+                try (Writer writer = new OutputStreamWriter(out, StandardCharsets.UTF_8)) {
+                    writer.write(typeName);
+
+                    writer.flush();
+                }
+            }
+            catch (IOException e) {
+                U.error(log, "Failed to write class name to file [platformId=" + platformId + "id=" + typeId +
+                        ", clsName=" + typeName + ", file=" + file.getAbsolutePath() + ']', e);
+            }
+            catch(OverlappingFileLockException ignored) {
+                if (log.isDebugEnabled())
+                    log.debug("File already locked (will ignore): " + file.getAbsolutePath());
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                U.error(log, "Interrupted while waiting for acquiring file lock: " + file, e);
+            }
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     */
+    String readMapping(byte platformId, int typeId) throws IgniteCheckedException {
+        String fileName = getFileName(platformId, typeId);
+
+        Lock lock = fileLock(fileName);
+
+        lock.lock();
+
+        try {
+            File file = new File(workDir, fileName);
+
+            try (FileInputStream in = new FileInputStream(file)) {
+                FileLock fileLock = fileLock(in.getChannel(), true);
+
+                assert fileLock != null : fileName;
+
+                try (BufferedReader reader = new BufferedReader(new InputStreamReader(in, StandardCharsets.UTF_8))) {
+                    return reader.readLine();
+                }
+            }
+            catch (IOException ignored) {
+                return null;
+            }
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param platformId Platform id.
+     * @param typeId Type id.
+     */
+    private String getFileName(byte platformId, int typeId) {
+        return typeId + ".classname" + platformId;
+    }
+
+    /**
+     * @param fileName File name.
+     * @return Lock instance.
+     */
+    private static Lock fileLock(String fileName) {
+        return fileLock.getLock(fileName.hashCode());
+    }
+
+    /**
+     * @param ch File channel.
+     * @param shared Shared.
+     */
+    private static FileLock fileLock(
+            FileChannel ch,
+            boolean shared
+    ) throws IOException, IgniteInterruptedCheckedException {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        while (true) {
+            FileLock fileLock = ch.tryLock(0L, Long.MAX_VALUE, shared);
+
+            if (fileLock == null)
+                U.sleep(rnd.nextLong(50));
+            else
+                return fileLock;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/MarshallerPlatformIds.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerPlatformIds.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerPlatformIds.java
new file mode 100644
index 0000000..458ae49
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerPlatformIds.java
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * Constants for platform IDs to feed into {@link org.apache.ignite.marshaller.MarshallerContext}.
+ */
+public final class MarshallerPlatformIds {
+    /** */
+    public static final byte JAVA_ID = 0;
+
+    /** */
+    private MarshallerPlatformIds() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/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
index 2237c27..b291872 100644
--- 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
@@ -118,6 +118,8 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.jar.JarEntry;
 import java.util.jar.JarFile;
 
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
+
 /**
  * Binary context.
  */
@@ -764,7 +766,7 @@ public class BinaryContext {
         final int typeId = mapper.typeId(clsName);
 
         try {
-            registered = marshCtx.registerClass(typeId, cls);
+            registered = marshCtx.registerClassName(JAVA_ID, typeId, cls.getName());
         }
         catch (IgniteCheckedException e) {
             throw new BinaryObjectException("Failed to register class.", e);
@@ -807,7 +809,7 @@ public class BinaryContext {
         boolean registered;
 
         try {
-            registered = marshCtx.registerClass(desc.typeId(), desc.describedClass());
+            registered = marshCtx.registerClassName(JAVA_ID, desc.typeId(), desc.describedClass().getName());
         }
         catch (IgniteCheckedException e) {
             throw new BinaryObjectException("Failed to register class.", e);
@@ -844,7 +846,7 @@ public class BinaryContext {
      * @param cls Class.
      * @return Serializer for class or {@code null} if none exists.
      */
-    private @Nullable BinarySerializer serializerForClass(Class cls) {
+    @Nullable private BinarySerializer serializerForClass(Class cls) {
         BinarySerializer serializer = defaultSerializer();
 
         if (serializer == null && canUseReflectiveSerializer(cls))

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
index 3c65db6..6a4b5e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/marshaller/optimized/GridClientOptimizedMarshaller.java
@@ -22,7 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.MarshallerContextAdapter;
+import org.apache.ignite.internal.MarshallerContextImpl;
 import org.apache.ignite.internal.client.marshaller.GridClientMarshaller;
 import org.apache.ignite.internal.processors.rest.client.message.GridClientMessage;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -31,7 +31,7 @@ import org.apache.ignite.plugin.PluginProvider;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Wrapper, that adapts {@link org.apache.ignite.marshaller.optimized.OptimizedMarshaller} to
+ * Wrapper, that adapts {@link OptimizedMarshaller} to
  * {@link GridClientMarshaller} interface.
  */
 public class GridClientOptimizedMarshaller implements GridClientMarshaller {
@@ -114,7 +114,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
 
     /**
      */
-    private static class ClientMarshallerContext extends MarshallerContextAdapter {
+    private static class ClientMarshallerContext extends MarshallerContextImpl {
         /** */
         public ClientMarshallerContext() {
             super(null);
@@ -126,15 +126,5 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
         public ClientMarshallerContext(@Nullable List<PluginProvider> plugins) {
             super(plugins);
         }
-
-        /** {@inheritDoc} */
-        @Override protected boolean registerClassName(int id, String clsName) {
-            throw new UnsupportedOperationException(clsName);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected String className(int id) {
-            throw new UnsupportedOperationException();
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 584cc56..59af748 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -59,6 +59,9 @@ import org.apache.ignite.spi.IgniteSpiContext;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.IgniteSpiNoop;
 import org.apache.ignite.spi.IgniteSpiTimeoutObject;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.jetbrains.annotations.Nullable;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
@@ -604,12 +607,22 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
-        return null;
+    @Override public void collectJoiningNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void collectGridNodeData(DiscoveryDataBag dataBag) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onGridDataReceived(GridDiscoveryData data) {
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable data) {
+    @Override public void onJoiningNodeDataReceived(JoiningNodeDiscoveryData data) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index de34adb..16ea972 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -94,7 +94,6 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.DAT
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MANAGEMENT_POOL;
-import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.P2P_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
@@ -684,7 +683,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 case MANAGEMENT_POOL:
                 case AFFINITY_POOL:
                 case UTILITY_CACHE_POOL:
-                case MARSH_CACHE_POOL:
                 case IDX_POOL:
                 case IGFS_POOL:
                 case DATA_STREAMER_POOL:

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index b1fe910..e283bdc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -131,6 +131,8 @@ import org.apache.ignite.internal.processors.igfs.IgfsFileAffinityRange;
 import org.apache.ignite.internal.processors.igfs.IgfsFragmentizerRequest;
 import org.apache.ignite.internal.processors.igfs.IgfsFragmentizerResponse;
 import org.apache.ignite.internal.processors.igfs.IgfsSyncMessage;
+import org.apache.ignite.internal.processors.marshaller.MissingMappingRequestMessage;
+import org.apache.ignite.internal.processors.marshaller.MissingMappingResponseMessage;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse;
 import org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest;
@@ -806,6 +808,16 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
+            case 120:
+                msg = new MissingMappingRequestMessage();
+
+                break;
+
+            case 121:
+                msg = new MissingMappingResponseMessage();
+
+                break;
+
             case 124:
                 msg = new GridMessageCollection<>();
 


[2/5] ignite git commit: IGNITE-4157 Use discovery custom messages instead of marshaller cache - Fixes #1271.

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
index bd52c04..cdd1f2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
@@ -17,16 +17,14 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
-import java.util.Arrays;
 import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.UUID;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.jetbrains.annotations.Nullable;
 
@@ -44,6 +42,9 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
     /** Added node. */
     private final TcpDiscoveryNode node;
 
+    /** */
+    private DiscoveryDataPacket dataPacket;
+
     /** Pending messages from previous node. */
     private Collection<TcpDiscoveryAbstractMessage> msgs;
 
@@ -64,12 +65,6 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
     /** Topology snapshots history. */
     private Map<Long, Collection<ClusterNode>> topHist;
 
-    /** Discovery data from new node. */
-    private Map<Integer, byte[]> newNodeDiscoData;
-
-    /** Discovery data from old nodes. */
-    private Map<UUID, Map<Integer, byte[]>> oldNodesDiscoData;
-
     /** Start time of the first grid node. */
     private final long gridStartTime;
 
@@ -78,12 +73,12 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
      *
      * @param creatorNodeId Creator node ID.
      * @param node Node to add to topology.
-     * @param newNodeDiscoData New Node discovery data.
+     * @param dataPacket container for collecting discovery data across the cluster.
      * @param gridStartTime Start time of the first grid node.
      */
     public TcpDiscoveryNodeAddedMessage(UUID creatorNodeId,
         TcpDiscoveryNode node,
-        Map<Integer, byte[]> newNodeDiscoData,
+        DiscoveryDataPacket dataPacket,
         long gridStartTime)
     {
         super(creatorNodeId);
@@ -92,10 +87,8 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
         assert gridStartTime > 0;
 
         this.node = node;
-        this.newNodeDiscoData = newNodeDiscoData;
+        this.dataPacket = dataPacket;
         this.gridStartTime = gridStartTime;
-
-        oldNodesDiscoData = new LinkedHashMap<>();
     }
 
     /**
@@ -111,8 +104,7 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
         this.top = msg.top;
         this.clientTop = msg.clientTop;
         this.topHist = msg.topHist;
-        this.newNodeDiscoData = msg.newNodeDiscoData;
-        this.oldNodesDiscoData = msg.oldNodesDiscoData;
+        this.dataPacket = msg.dataPacket;
         this.gridStartTime = msg.gridStartTime;
     }
 
@@ -222,63 +214,17 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
-     * @return Discovery data from new node.
-     */
-    public Map<Integer, byte[]> newNodeDiscoveryData() {
-        return newNodeDiscoData;
-    }
-
-    /**
-     * @return Discovery data from old nodes.
+     * @return {@link DiscoveryDataPacket} carried by this message.
      */
-    public Map<UUID, Map<Integer, byte[]>> oldNodesDiscoveryData() {
-        return oldNodesDiscoData;
-    }
-
-    /**
-     * @param oldNodesDiscoData Discovery data from old nodes.
-     */
-    public void oldNodesDiscoveryData(Map<UUID, Map<Integer, byte[]>> oldNodesDiscoData) {
-        this.oldNodesDiscoData = oldNodesDiscoData;
-    }
-
-    /**
-     * @param nodeId Node ID.
-     * @param discoData Discovery data to add.
-     */
-    public void addDiscoveryData(UUID nodeId, Map<Integer, byte[]> discoData) {
-        // Old nodes disco data may be null if message
-        // makes more than 1 pass due to stopping of the nodes in topology.
-        if (oldNodesDiscoData != null) {
-            for (Map.Entry<UUID, Map<Integer, byte[]>> existingDataEntry : oldNodesDiscoData.entrySet()) {
-                Map<Integer, byte[]> existingData = existingDataEntry.getValue();
-
-                Iterator<Map.Entry<Integer, byte[]>> it = discoData.entrySet().iterator();
-
-                while (it.hasNext()) {
-                    Map.Entry<Integer, byte[]> discoDataEntry = it.next();
-
-                    byte[] curData = existingData.get(discoDataEntry.getKey());
-
-                    if (Arrays.equals(curData, discoDataEntry.getValue()))
-                        it.remove();
-                }
-
-                if (discoData.isEmpty())
-                    break;
-            }
-
-            if (!discoData.isEmpty())
-                oldNodesDiscoData.put(nodeId, discoData);
-        }
+    public DiscoveryDataPacket gridDiscoveryData() {
+        return dataPacket;
     }
 
     /**
      * Clears discovery data to minimize message size.
      */
     public void clearDiscoveryData() {
-        newNodeDiscoData = null;
-        oldNodesDiscoData = null;
+        dataPacket = null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
index 38685f2..2d7eca4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
@@ -118,7 +118,7 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest {
     private Collection<CacheConfiguration> caches(Ignite g) {
         return F.view(Arrays.asList(g.configuration().getCacheConfiguration()), new IgnitePredicate<CacheConfiguration>() {
             @Override public boolean apply(CacheConfiguration c) {
-                return !CU.MARSH_CACHE_NAME.equals(c.getName()) && !CU.UTILITY_CACHE_NAME.equals(c.getName()) &&
+                return !CU.UTILITY_CACHE_NAME.equals(c.getName()) &&
                     !CU.ATOMICS_CACHE_NAME.equals(c.getName()) && !CU.SYS_CACHE_HADOOP_MR.equals(c.getName());
             }
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java
index bf0adf8..cc2bc39 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/MarshallerContextLockingSelfTest.java
@@ -17,37 +17,49 @@
 
 package org.apache.ignite.internal;
 
-import java.io.File;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicInteger;
-import javax.cache.event.CacheEntryEvent;
-import javax.cache.event.EventType;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
-import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem;
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingTransport;
 import org.apache.ignite.testframework.GridTestClassLoader;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
 
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
+
 /**
  * Test marshaller context.
  */
 public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest {
     /** Inner logger. */
-    private InnerLogger innerLog = null;
+    private InnerLogger innerLog;
+
+    private GridTestKernalContext ctx;
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         innerLog = new InnerLogger();
 
-        log = innerLog;
+        IgniteConfiguration iCfg = new IgniteConfiguration();
+        iCfg.setClientMode(false);
+
+        ctx = new GridTestKernalContext(innerLog, iCfg) {
+            @Override public IgniteLogger log(Class<?> cls) {
+                return innerLog;
+            }
+        };
+
+        ctx.setSystemExecutorService(Executors.newFixedThreadPool(12));
     }
 
     /**
-     * Mumtithread test, used custom class loader
+     * Multithreaded test, used custom class loader
      */
     public void testMultithreadedUpdate() throws Exception {
         multithreaded(new Callable<Object>() {
@@ -55,7 +67,10 @@ public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest {
                 GridTestClassLoader classLoader = new GridTestClassLoader(
                     InternalExecutor.class.getName(),
                     MarshallerContextImpl.class.getName(),
-                    MarshallerContextImpl.ContinuousQueryListener.class.getName()
+                    MarshallerContextImpl.CombinedMap.class.getName(),
+                    MappingStoreTask.class.getName(),
+                    MarshallerMappingFileStore.class.getName(),
+                    MarshallerMappingTransport.class.getName()
                 );
 
                 Thread.currentThread().setContextClassLoader(classLoader);
@@ -64,7 +79,8 @@ public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest {
 
                 Object internelExecutor = clazz.newInstance();
 
-                clazz.getMethod("executeTest", GridTestLog4jLogger.class).invoke(internelExecutor, log);
+                clazz.getMethod("executeTest", GridTestLog4jLogger.class, GridKernalContext.class)
+                        .invoke(internelExecutor, log, ctx);
 
                 return null;
             }
@@ -72,9 +88,9 @@ public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest {
 
         assertTrue(InternalExecutor.counter.get() == 0);
 
-        assertTrue(innerLog.contains("File already locked"));
-
         assertTrue(!innerLog.contains("Exception"));
+
+        assertTrue(innerLog.contains("File already locked"));
     }
 
     /**
@@ -87,21 +103,16 @@ public class MarshallerContextLockingSelfTest extends GridCommonAbstractTest {
         /**
         * Executes onUpdated
         */
-        public void executeTest(GridTestLog4jLogger log) throws Exception {
+        public void executeTest(GridTestLog4jLogger log, GridKernalContext ctx) throws Exception {
             counter.incrementAndGet();
 
-            File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false);
-
-            final MarshallerContextImpl.ContinuousQueryListener queryListener = new MarshallerContextImpl.ContinuousQueryListener(log, workDir);
-
-            final ArrayList evts = new ArrayList<CacheEntryEvent<Integer, String>>();
-
-            IgniteCacheProxy cache = new IgniteCacheProxy();
+            MarshallerContextImpl marshallerContext = new MarshallerContextImpl(null);
+            marshallerContext.onMarshallerProcessorStarted(ctx, null);
 
-            evts.add(new CacheContinuousQueryManager.CacheEntryEventImpl(cache, EventType.CREATED, 1, String.class.getName()));
+            MarshallerMappingItem item = new MarshallerMappingItem(JAVA_ID, 1, String.class.getName());
 
-            for (int i = 0; i < 100; i++)
-                queryListener.onUpdated(evts);
+            for (int i = 0; i < 400; i++)
+                marshallerContext.onMappingAccepted(item);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
index cd8a487..003a330 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
@@ -53,6 +53,7 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentSkipListSet;
 import junit.framework.Assert;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryBasicIdMapper;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
@@ -75,6 +76,7 @@ import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.IgniteUtils;
@@ -84,7 +86,10 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -3564,12 +3569,25 @@ public class BinaryMarshallerSelfTest extends GridCommonAbstractTest {
         bCfg.setTypeConfigurations(cfgs);
 
         iCfg.setBinaryConfiguration(bCfg);
+        iCfg.setClientMode(false);
+        iCfg.setDiscoverySpi(new TcpDiscoverySpi() {
+            @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+                //No-op.
+            }
+        });
 
         BinaryContext ctx = new BinaryContext(BinaryCachingMetadataHandler.create(), iCfg, new NullLogger());
 
         BinaryMarshaller marsh = new BinaryMarshaller();
 
-        marsh.setContext(new MarshallerContextTestImpl(null, excludedClasses));
+        MarshallerContextTestImpl marshCtx = new MarshallerContextTestImpl(null, excludedClasses);
+
+        GridTestKernalContext kernCtx = new GridTestKernalContext(log, iCfg);
+        kernCtx.add(new GridDiscoveryManager(kernCtx));
+
+        marshCtx.onMarshallerProcessorStarted(kernCtx, null);
+
+        marsh.setContext(marshCtx);
 
         IgniteUtils.invoke(BinaryMarshaller.class, marsh, "setBinaryContext", ctx, iCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
index 2b0051e..5ee2423 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/binary/GridBinaryMarshallerCtxDisabledSelfTest.java
@@ -17,23 +17,22 @@
 
 package org.apache.ignite.internal.binary;
 
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Arrays;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryReader;
 import org.apache.ignite.binary.BinaryWriter;
 import org.apache.ignite.binary.Binarylizable;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.MarshallerContextAdapter;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.logger.NullLogger;
+import org.apache.ignite.marshaller.MarshallerContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.util.Arrays;
-
 /**
  *
  */
@@ -88,21 +87,36 @@ public class GridBinaryMarshallerCtxDisabledSelfTest extends GridCommonAbstractT
      * Marshaller context with no storage. Platform has to work in such environment as well by marshalling class name of
      * a binary object.
      */
-    private static class MarshallerContextWithNoStorage extends MarshallerContextAdapter {
-        /** */
-        public MarshallerContextWithNoStorage() {
-            super(null);
+    private static class MarshallerContextWithNoStorage implements MarshallerContext {
+        /** {@inheritDoc} */
+        @Override public boolean registerClassName(
+                byte platformId,
+                int typeId,
+                String clsName
+        ) throws IgniteCheckedException {
+            return false;
         }
 
         /** {@inheritDoc} */
-        @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
-            return false;
+        @Override public Class getClass(
+                int typeId,
+                ClassLoader ldr
+        ) throws ClassNotFoundException, IgniteCheckedException {
+            return null;
         }
 
         /** {@inheritDoc} */
-        @Override protected String className(int id) throws IgniteCheckedException {
+        @Override public String getClassName(
+                byte platformId,
+                int typeId
+        ) throws ClassNotFoundException, IgniteCheckedException {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public boolean isSystemType(String typeName) {
+            return false;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java
index 9e3ea7b..fd2dd80 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryMemorySizeSelfTest.java
@@ -137,11 +137,15 @@ public class GridCacheEntryMemorySizeSelfTest extends GridCommonAbstractTest {
         Marshaller marsh = new OptimizedMarshaller();
 
         marsh.setContext(new MarshallerContext() {
-            @Override public boolean registerClass(int id, Class cls) {
+            @Override public boolean registerClassName(byte platformId, int typeId, String clsName) {
                 return true;
             }
 
-            @Override public Class getClass(int id, ClassLoader ldr) {
+            @Override public Class getClass(int typeId, ClassLoader ldr) {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override public String getClassName(byte platformId, int typeId) {
                 throw new UnsupportedOperationException();
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteInternalCacheTypesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteInternalCacheTypesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteInternalCacheTypesTest.java
index ea7b124..8255dc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteInternalCacheTypesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteInternalCacheTypesTest.java
@@ -29,7 +29,6 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
-import static org.apache.ignite.internal.managers.communication.GridIoPolicy.MARSH_CACHE_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.UTILITY_CACHE_POOL;
 
@@ -113,14 +112,6 @@ public class IgniteInternalCacheTypesTest extends GridCommonAbstractTest {
 
         GridTestUtils.assertThrows(log(), new Callable<Object>() {
             @Override public Object call() throws Exception {
-                ignite.cache(CU.MARSH_CACHE_NAME);
-
-                return null;
-            }
-        }, IllegalStateException.class, null);
-
-        GridTestUtils.assertThrows(log(), new Callable<Object>() {
-            @Override public Object call() throws Exception {
                 ignite.cache(CU.ATOMICS_CACHE_NAME);
 
                 return null;
@@ -129,8 +120,6 @@ public class IgniteInternalCacheTypesTest extends GridCommonAbstractTest {
 
         checkCache(ignite, CU.UTILITY_CACHE_NAME, UTILITY_CACHE_POOL, false, true);
 
-        checkCache(ignite, CU.MARSH_CACHE_NAME, MARSH_CACHE_POOL, false, false);
-
         checkCache(ignite, CU.ATOMICS_CACHE_NAME, SYSTEM_POOL, false, true);
 
         for (String cache : userCaches)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
new file mode 100644
index 0000000..50107e4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClassNameConflictTest.java
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.cache;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.DiscoverySpiListener;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Tests situation when two nodes in cluster simultaneously propose different classes with the same typeId
+ * (which is actually class name's <b>hashCode</b> ).
+ *
+ * In that case one of the propose requests should be rejected
+ * and {@link org.apache.ignite.internal.processors.marshaller.MappingProposedMessage} is sent
+ * with not-null <b>conflictingClsName</b> field.
+ */
+public class IgniteMarshallerCacheClassNameConflictTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private volatile boolean bbClsRejected;
+
+    /** */
+    private volatile boolean aaClsRejected;
+
+    /** */
+    private volatile boolean rejectObserved;
+
+    /**
+     * Latch used to synchronize two nodes on sending mapping requests for classes with conflicting names.
+     */
+    private static final CountDownLatch startLatch = new CountDownLatch(3);
+
+    /** */
+    private static volatile boolean busySpinFlag;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TestTcpDiscoverySpi();
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCachePutGetClassesWithNameConflict() throws Exception {
+        Ignite srv1 = startGrid(0);
+        Ignite srv2 = startGrid(1);
+        ExecutorService exec1 = srv1.executorService();
+        ExecutorService exec2 = srv2.executorService();
+
+        final AtomicInteger trickCompilerVar = new AtomicInteger(1);
+
+        final Organization aOrg1 = new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA");
+        final OrganizatioN bOrg2 = new OrganizatioN(2, "Apple", "1 Infinite Loop, Cupertino, CA 95014, USA");
+
+        exec1.submit(new Runnable() {
+            @Override public void run() {
+                startLatch.countDown();
+
+                try {
+                    startLatch.await();
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+
+                //busy spinning after waking up from startLatch.await
+                // to reduce probability that one thread starts significantly earlier than the other
+                while (!busySpinFlag) {
+                    if (trickCompilerVar.get() < 0)
+                        break;
+                }
+
+                Ignition.localIgnite().cache(null).put(1, aOrg1);
+            }
+        });
+
+        exec2.submit(new Runnable() {
+            @Override public void run() {
+                startLatch.countDown();
+
+                try {
+                    startLatch.await();
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+
+                //busy spinning after waking up from startLatch.await
+                // to reduce probability that one thread starts significantly earlier than the other
+                while (!busySpinFlag) {
+                    if (trickCompilerVar.get() < 0)
+                        break;
+                }
+
+                Ignition.localIgnite().cache(null).put(2, bOrg2);
+            }
+        });
+        startLatch.countDown();
+
+        busySpinFlag = true;
+
+        exec1.shutdown();
+        exec2.shutdown();
+
+        exec1.awaitTermination(100, TimeUnit.MILLISECONDS);
+        exec2.awaitTermination(100, TimeUnit.MILLISECONDS);
+
+        Ignite ignite = startGrid(2);
+
+        int cacheSize = ignite.cache(null).size(CachePeekMode.PRIMARY);
+
+        assertTrue("Expected cache size 1 but was " + cacheSize, cacheSize == 1);
+
+        if (rejectObserved)
+            assertTrue(aaClsRejected || bbClsRejected);
+    }
+
+    /** */
+    private class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+
+        /** */
+        private class DiscoverySpiListenerWrapper implements DiscoverySpiListener {
+            /** */
+            private DiscoverySpiListener delegate;
+
+            /**
+             * @param delegate Delegate.
+             */
+            private DiscoverySpiListenerWrapper(DiscoverySpiListener delegate) {
+                this.delegate = delegate;
+            }
+
+            /** {@inheritDoc} */
+            @Override public void onDiscovery(
+                    int type,
+                    long topVer,
+                    ClusterNode node,
+                    Collection<ClusterNode> topSnapshot,
+                    @Nullable Map<Long, Collection<ClusterNode>> topHist,
+                    @Nullable DiscoverySpiCustomMessage spiCustomMsg
+            ) {
+                DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
+                        : (DiscoveryCustomMessage) U.field(spiCustomMsg, "delegate");
+
+                if (customMsg != null)
+                    //don't want to make this class public, using equality of class name instead of instanceof operator
+                    if ("MappingProposedMessage".equals(customMsg.getClass().getSimpleName())) {
+                        String conflClsName = U.field(customMsg, "conflictingClsName");
+                        if (conflClsName != null && !conflClsName.isEmpty()) {
+                            rejectObserved = true;
+                            if (conflClsName.contains("Organization"))
+                                bbClsRejected = true;
+                            else if (conflClsName.contains("OrganizatioN"))
+                                aaClsRejected = true;
+                        }
+                    }
+
+                delegate.onDiscovery(type, topVer, node, topSnapshot, topHist, spiCustomMsg);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
+            super.setListener(new DiscoverySpiListenerWrapper(lsnr));
+        }
+    }
+
+    /**
+     * Class name is chosen to be in conflict with other class name this test put to cache.
+     */
+    private static class Organization {
+        /** */
+        private final int id;
+
+        /** */
+        private final String name;
+
+        /** */
+        private final String addr;
+
+        /**
+         * @param id Id.
+         * @param name Name.
+         * @param addr Address.
+         */
+        Organization(int id, String name, String addr) {
+            this.id = id;
+            this.name = name;
+            this.addr = addr;
+        }
+    }
+
+    /**
+     * Class name is chosen to be in conflict with other class name this test put to cache.
+     */
+    private static class OrganizatioN {
+        /** */
+        private final int id;
+
+        /** */
+        private final String name;
+
+        /** */
+        private final String addr;
+
+        /**
+         * @param id Id.
+         * @param name Name.
+         * @param addr Address.
+         */
+        OrganizatioN(int id, String name, String addr) {
+            this.id = id;
+            this.name = name;
+            this.addr = addr;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
new file mode 100644
index 0000000..f1c1b3a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteMarshallerCacheClientRequestsMappingOnMissTest.java
@@ -0,0 +1,345 @@
+/*
+ * 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.processors.cache;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridComponent;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridMessageListener;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Tests for client requesting missing mappings from server nodes with and without server nodes failures.
+ */
+public class IgniteMarshallerCacheClientRequestsMappingOnMissTest extends GridCommonAbstractTest {
+    /**
+     * Need to point client node to a different working directory
+     * to avoid reading marshaller mapping from FS and to force sending MissingMappingRequest.
+     */
+    private static final String TMP_DIR = System.getProperty("java.io.tmpdir");
+
+    /** */
+    private static final AtomicInteger mappingReqsCounter = new AtomicInteger(0);
+
+    /** */
+    private TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean clientMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(clientMode);
+
+        if (clientMode)
+            cfg.setWorkDirectory(TMP_DIR);
+
+        TcpDiscoverySpi disco = new TestTcpDiscoverySpi();
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanupMarshallerFileStore();
+
+        mappingReqsCounter.set(0);
+    }
+
+    /**
+     *
+     */
+    private void cleanupMarshallerFileStore() throws IOException {
+        Path marshCache = Paths.get(TMP_DIR, "marshaller");
+
+        for (File file : marshCache.toFile().listFiles())
+            Files.delete(file.toPath());
+
+        Files.deleteIfExists(marshCache);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRequestedMappingIsStoredInFS() throws Exception {
+        Ignite srv1 = startGrid(0);
+
+        Organization org = new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA");
+
+        srv1.cache(null).put(1, org);
+
+        clientMode = true;
+
+        Ignite cl1 = startGrid(1);
+
+        cl1.cache(null).get(1);
+
+        String clsName = Organization.class.getName();
+
+        stopGrid(1);
+
+        if (!getMarshCtxFileStoreExecutorSrvc((GridKernalContext) U.field(cl1, "ctx"))
+                .awaitTermination(5000, TimeUnit.MILLISECONDS))
+            fail("Failed to wait for executor service used by MarshallerContext to shutdown");
+
+        File[] files = Paths.get(TMP_DIR, "marshaller").toFile().listFiles();
+
+        assertNotNull(TMP_DIR + "/marshaller directory should contain at least one file", files);
+        assertEquals(TMP_DIR + "/marshaller directory should contain exactly one file", 1, files.length);
+        assertEquals(clsName, new String(Files.readAllBytes(files[0].toPath())));
+    }
+
+    /**
+     * @param ctx Context.
+     */
+    private ExecutorService getMarshCtxFileStoreExecutorSrvc(GridKernalContext ctx) {
+        return U.field(U.field(ctx, "marshCtx"), "execSrvc");
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoNodesDieOnRequest() throws Exception {
+        Ignite srv1 = startGrid(0);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv1, "ctx")).io());
+
+        Ignite srv2 = startGrid(1);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv2, "ctx")).io());
+
+        Ignite srv3 = startGrid(2);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv3, "ctx")).io());
+
+        srv3.cache(null).put(1, new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA"));
+
+        clientMode = true;
+
+        Ignite cl1 = startGrid(4);
+        cl1.cache(null).get(1);
+
+        assertEquals("Expected requests count is 1, actual is " + mappingReqsCounter.get(),
+                1,
+                mappingReqsCounter.get());
+    }
+
+    /**
+     *
+     */
+    public void testOneNodeDiesOnRequest() throws Exception {
+        Ignite srv1 = startGrid(0);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv1, "ctx")).io(), 0);
+
+        Ignite srv2 = startGrid(1);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv2, "ctx")).io());
+
+        Ignite srv3 = startGrid(2);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv3, "ctx")).io());
+
+        srv3.cache(null).put(1, new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA"));
+
+        clientMode = true;
+
+        Ignite cl1 = startGrid(4);
+        cl1.cache(null).get(1);
+
+        assertEquals("Expected requests count is 2, actual is " + mappingReqsCounter.get(),
+                2,
+                mappingReqsCounter.get());
+    }
+
+    /**
+     *
+     */
+    public void testTwoNodesDieOnRequest() throws Exception {
+        Ignite srv1 = startGrid(0);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv1, "ctx")).io(), 0);
+
+        Ignite srv2 = startGrid(1);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv2, "ctx")).io(), 1);
+
+        Ignite srv3 = startGrid(2);
+        replaceWithCountingMappingRequestListener(((GridKernalContext)U.field(srv3, "ctx")).io());
+
+        srv3.cache(null).put(1, new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA"));
+
+        clientMode = true;
+
+        Ignite cl1 = startGrid(4);
+        cl1.cache(null).get(1);
+
+        assertEquals("Expected requests count is 3, actual is " + mappingReqsCounter.get(),
+                3,
+                mappingReqsCounter.get());
+    }
+
+    /**
+     *
+     */
+    public void testAllNodesDieOnRequest() throws Exception {
+        Ignite srv1 = startGrid(0);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv1, "ctx")).io(), 0);
+
+        Ignite srv2 = startGrid(1);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv2, "ctx")).io(), 1);
+
+        Ignite srv3 = startGrid(2);
+        replaceWithStoppingMappingRequestListener(((GridKernalContext)U.field(srv3, "ctx")).io(), 2);
+
+        srv3.cache(null).put(1, new Organization(1, "Microsoft", "One Microsoft Way Redmond, WA 98052-6399, USA"));
+
+        clientMode = true;
+
+        Ignite cl1 = startGrid(4);
+        try {
+            cl1.cache(null).get(1);
+        }
+        catch (Exception e) {
+            e.printStackTrace();
+        }
+
+        assertEquals("Expected requests count is 3, actual is " + mappingReqsCounter.get(),
+                3,
+                mappingReqsCounter.get());
+    }
+
+    /**
+     *
+     */
+    private void replaceWithCountingMappingRequestListener(GridIoManager ioMgr) {
+        GridMessageListener[] lsnrs = U.field(ioMgr, "sysLsnrs");
+
+        final GridMessageListener delegate = lsnrs[GridTopic.TOPIC_MAPPING_MARSH.ordinal()];
+
+        GridMessageListener wrapper = new GridMessageListener() {
+            @Override public void onMessage(UUID nodeId, Object msg) {
+                mappingReqsCounter.incrementAndGet();
+                delegate.onMessage(nodeId, msg);
+            }
+        };
+
+        lsnrs[GridTopic.TOPIC_MAPPING_MARSH.ordinal()] = wrapper;
+    }
+
+    /**
+     *
+     */
+    private void replaceWithStoppingMappingRequestListener(GridIoManager ioMgr, final int nodeIdToStop) {
+        ioMgr.removeMessageListener(GridTopic.TOPIC_MAPPING_MARSH);
+
+        ioMgr.addMessageListener(GridTopic.TOPIC_MAPPING_MARSH, new GridMessageListener() {
+            @Override public void onMessage(UUID nodeId, Object msg) {
+                new Thread(new Runnable() {
+                    @Override public void run() {
+                        mappingReqsCounter.incrementAndGet();
+                        stopGrid(nodeIdToStop, true);
+                    }
+                }).start();
+            }
+        });
+    }
+
+    /**
+     *
+     */
+    private static class Organization {
+        /** */
+        private final int id;
+
+        /** */
+        private final String name;
+
+        /** */
+        private final String addr;
+
+        /**
+         * @param id Id.
+         * @param name Name.
+         * @param addr Address.
+         */
+        Organization(int id, String name, String addr) {
+            this.id = id;
+            this.name = name;
+            this.addr = addr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return "Organization{" +
+                    "id=" + id +
+                    ", name='" + name + '\'' +
+                    ", addr='" + addr + '\'' +
+                    '}';
+        }
+    }
+
+    /**
+     * This implementation prevents client nodes from obtaining marshaller mapping data on discovery phase.
+     *
+     * It is needed to force client to request mapping from grid.
+     */
+    private static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** {@inheritDoc} */
+        @Override protected void onExchange(DiscoveryDataPacket dataPacket, ClassLoader clsLdr) {
+            if (locNode.isClient()) {
+                Map<Integer, byte[]> cmnData = U.field(dataPacket, "commonData");
+
+                cmnData.remove(GridComponent.DiscoveryDataExchangeType.MARSHALLER_PROC.ordinal());
+            }
+
+            super.onExchange(dataPacket, clsLdr);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
index 66356cb..8545f82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
@@ -65,30 +65,9 @@ public class IgniteSystemCacheOnClientTest extends GridCommonAbstractTest {
 
         assertTrue(ignite.configuration().isClientMode());
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                return ignite.internalCache(CU.MARSH_CACHE_NAME) != null;
-            }
-        }, 5000);
-
-        GridCacheAdapter marshCache = ignite.internalCache(CU.MARSH_CACHE_NAME);
-
-        assertNotNull(marshCache);
-
-        assertFalse(marshCache.context().isNear());
-
-        marshCache = ((IgniteKernal)ignite(0)).internalCache(CU.MARSH_CACHE_NAME);
-
-        assertFalse(marshCache.context().isNear());
-
-        Collection<ClusterNode> affNodes = marshCache.affinity().mapKeyToPrimaryAndBackups(1);
-
-        assertEquals(1, affNodes.size());
-        assertTrue(affNodes.contains(ignite(0).cluster().localNode()));
-
         GridCacheAdapter utilityCache = ((IgniteKernal)ignite(0)).internalCache(CU.UTILITY_CACHE_NAME);
 
-        affNodes = utilityCache.affinity().mapKeyToPrimaryAndBackups(1);
+        Collection<ClusterNode> affNodes = utilityCache.affinity().mapKeyToPrimaryAndBackups(1);
 
         assertEquals(1, affNodes.size());
         assertTrue(affNodes.contains(ignite(0).cluster().localNode()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
index c7a6a55..3571d08 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridBinaryCacheEntryMemorySizeSelfTest.java
@@ -18,15 +18,20 @@
 package org.apache.ignite.internal.processors.cache.binary;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.binary.BinaryNoopMetadataHandler;
 import org.apache.ignite.internal.binary.BinaryContext;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryMemorySizeSelfTest;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.logger.NullLogger;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
 
 /**
  *
@@ -36,9 +41,21 @@ public class GridBinaryCacheEntryMemorySizeSelfTest extends GridCacheEntryMemory
     @Override protected Marshaller createMarshaller() throws IgniteCheckedException {
         BinaryMarshaller marsh = new BinaryMarshaller();
 
-        marsh.setContext(new MarshallerContextTestImpl(null));
-
         IgniteConfiguration iCfg = new IgniteConfiguration();
+        iCfg.setDiscoverySpi(new TcpDiscoverySpi() {
+            @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+                // No-op.
+            }
+        });
+        iCfg.setClientMode(false);
+
+        GridTestKernalContext kernCtx = new GridTestKernalContext(log, iCfg);
+        kernCtx.add(new GridDiscoveryManager(kernCtx));
+
+        MarshallerContextTestImpl marshCtx = new MarshallerContextTestImpl(null);
+        marshCtx.onMarshallerProcessorStarted(kernCtx, null);
+
+        marsh.setContext(marshCtx);
 
         BinaryContext pCtx = new BinaryContext(BinaryNoopMetadataHandler.instance(), iCfg, new NullLogger());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
index 489a402..09804c7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
@@ -59,7 +59,7 @@ public class IgniteCacheSystemTransactionsSelfTest extends GridCacheAbstractSelf
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        for (String cacheName : new String[] {null, CU.UTILITY_CACHE_NAME, CU.MARSH_CACHE_NAME}) {
+        for (String cacheName : new String[] {null, CU.UTILITY_CACHE_NAME}) {
             IgniteKernal kernal = (IgniteKernal)ignite(0);
 
             GridCacheAdapter<Object, Object> cache = kernal.context().cache().internalCache(cacheName);
@@ -108,22 +108,6 @@ public class IgniteCacheSystemTransactionsSelfTest extends GridCacheAbstractSelf
     /**
      * @throws Exception If failed.
      */
-    public void testMarshallerCacheShouldNotStartTx() throws Exception {
-        IgniteKernal ignite = (IgniteKernal)grid(0);
-
-        final GridCacheAdapter<String,String> marshallerCache = (GridCacheAdapter<String, String>)(GridCacheAdapter)
-            ignite.context().cache().marshallerCache();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return marshallerCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
-            }
-        }, IgniteException.class, null);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     private void checkTransactionsCommitted() throws Exception {
         for (int i = 0; i < gridCount(); i++) {
             IgniteKernal kernal = (IgniteKernal)grid(i);

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 34b67bc..7fed37d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -159,12 +159,12 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
         final int size = 10;
 
-        IgniteCache<Object, Object> chache0 = grid(0).cache(null);
+        IgniteCache<Object, Object> cache0 = grid(0).cache(null);
 
         for (int i = 0; i < size; i++) {
             info("Putting value [i=" + i + ']');
 
-            chache0.put(i, i);
+            cache0.put(i, i);
 
             info("Finished putting value [i=" + i + ']');
         }
@@ -175,13 +175,13 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         for (int i = 0; i < size; i++) {
             info("Putting value 2 [i=" + i + ']');
 
-            assertEquals(i, chache0.getAndPutIfAbsent(i, i * i));
+            assertEquals(i, cache0.getAndPutIfAbsent(i, i * i));
 
             info("Finished putting value 2 [i=" + i + ']');
         }
 
         for (int i = 0; i < size; i++)
-            assertEquals(i, chache0.get(i));
+            assertEquals(i, cache0.get(i));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index d5dc557..0e08b01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -211,7 +211,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
         for (int i = 0; i < gridCount(); i++) {
             GridContinuousProcessor proc = grid(i).context().continuous();
 
-            assertEquals(String.valueOf(i), 2, ((Map)U.field(proc, "locInfos")).size());
+            assertEquals(String.valueOf(i), 1, ((Map)U.field(proc, "locInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "rmtInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "startFuts")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "stopFuts")).size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
index 8a0ff9a..7ce6ece 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextSelfTest.java
@@ -18,46 +18,57 @@
 package org.apache.ignite.marshaller;
 
 import java.io.File;
+import java.io.IOException;
+import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.ArrayList;
-import javax.cache.event.EventType;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.MarshallerContextImpl;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager;
+import org.apache.ignite.internal.processors.marshaller.MarshallerMappingItem;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import static java.nio.file.Files.readAllBytes;
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
 
 /**
  * Test marshaller context.
  */
 public class MarshallerContextSelfTest extends GridCommonAbstractTest {
+    /** */
+    private GridTestKernalContext ctx;
+
+    /** */
+    private ExecutorService execSvc;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        ctx = newContext();
+        execSvc = Executors.newSingleThreadExecutor();
+
+        ctx.setSystemExecutorService(execSvc);
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testClassName() throws Exception {
-        File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false);
+        MarshallerContextImpl ctx = new MarshallerContextImpl(null);
 
-        final MarshallerContextImpl.ContinuousQueryListener queryListener =
-                new MarshallerContextImpl.ContinuousQueryListener(log, workDir);
+        ctx.onMarshallerProcessorStarted(this.ctx, null);
 
-        final ArrayList evts = new ArrayList<>();
+        MarshallerMappingItem item = new MarshallerMappingItem(JAVA_ID, 1, String.class.getName());
 
-        IgniteCacheProxy cache = new IgniteCacheProxy();
-
-        evts.add(new CacheContinuousQueryManager.CacheEntryEventImpl(cache,
-            EventType.CREATED,
-            1,
-            String.class.getName()));
-
-        queryListener.onUpdated(evts);
+        ctx.onMappingAccepted(item);
 
         try (Ignite g1 = startGrid(1)) {
             MarshallerContextImpl marshCtx = ((IgniteKernal)g1).context().marshallerContext();
-            String clsName = marshCtx.className(1);
+            String clsName = marshCtx.getClassName(JAVA_ID, 1);
 
             assertEquals("java.lang.String", clsName);
         }
@@ -68,23 +79,100 @@ public class MarshallerContextSelfTest extends GridCommonAbstractTest {
      */
     public void testOnUpdated() throws Exception {
         File workDir = U.resolveWorkDirectory(U.defaultWorkDirectory(), "marshaller", false);
+        MarshallerContextImpl ctx = new MarshallerContextImpl(null);
+
+        ctx.onMarshallerProcessorStarted(this.ctx, null);
+
+        MarshallerMappingItem item1 = new MarshallerMappingItem(JAVA_ID, 1, String.class.getName());
+
+        ctx.onMappingAccepted(item1);
+
+        checkFileName("java.lang.String", Paths.get(workDir + "/1.classname0"));
+
+        MarshallerMappingItem item2 = new MarshallerMappingItem((byte) 2, 2, "Random.Class.Name");
+
+        ctx.onMappingProposed(item2);
+        ctx.onMappingAccepted(item2);
+
+        execSvc.shutdown();
+        if (execSvc.awaitTermination(1000, TimeUnit.MILLISECONDS))
+            checkFileName("Random.Class.Name", Paths.get(workDir + "/2.classname2"));
+        else
+            fail("Failed to wait for executor service to shutdown");
+    }
+
+    /**
+     * Tests that there is a null value inserted in allCaches list
+     * if platform ids passed to marshaller cache were not sequential (like 0, 2).
+     */
+    public void testCacheStructure0() throws Exception {
+        MarshallerContextImpl ctx = new MarshallerContextImpl(null);
+
+        ctx.onMarshallerProcessorStarted(this.ctx, null);
+
+        MarshallerMappingItem item1 = new MarshallerMappingItem(JAVA_ID, 1, String.class.getName());
 
-        final MarshallerContextImpl.ContinuousQueryListener queryListener =
-                new MarshallerContextImpl.ContinuousQueryListener(log, workDir);
+        ctx.onMappingAccepted(item1);
 
-        final ArrayList evts = new ArrayList<>();
+        MarshallerMappingItem item2 = new MarshallerMappingItem((byte) 2, 2, "Random.Class.Name");
 
-        IgniteCacheProxy cache = new IgniteCacheProxy();
+        ctx.onMappingProposed(item2);
 
-        evts.add(new CacheContinuousQueryManager.CacheEntryEventImpl(cache,
-            EventType.CREATED,
-            1,
-            String.class.getName()));
+        List list = U.field(ctx, "allCaches");
 
-        queryListener.onUpdated(evts);
+        assertNotNull("Mapping cache is null for platformId: 0" , list.get(0));
+        assertNull("Mapping cache is not null for platformId: 1", list.get(1));
+        assertNotNull("Mapping cache is null for platformId: 2", list.get(2));
 
-        String fileName = "1.classname";
+        boolean excObserved = false;
+        try {
+            list.get(3);
+        }
+        catch (ArrayIndexOutOfBoundsException ignored) {
+            excObserved = true;
+        }
+        assertTrue("ArrayIndexOutOfBoundsException had to be thrown", excObserved);
+    }
+
+    /**
+     * Tests that there are no null values in allCaches list
+     * if platform ids passed to marshaller context were sequential.
+     */
+    public void testCacheStructure1() throws Exception {
+        MarshallerContextImpl ctx = new MarshallerContextImpl(null);
+
+        ctx.onMarshallerProcessorStarted(this.ctx, null);
+
+        MarshallerMappingItem item1 = new MarshallerMappingItem(JAVA_ID, 1, String.class.getName());
+
+        ctx.onMappingAccepted(item1);
+
+        MarshallerMappingItem item2 = new MarshallerMappingItem((byte) 1, 2, "Random.Class.Name");
+
+        ctx.onMappingProposed(item2);
 
-        assertEquals("java.lang.String", new String(readAllBytes(Paths.get(workDir + "/" + fileName))));
+        List list = U.field(ctx, "allCaches");
+
+        assertNotNull("Mapping cache is null for platformId: 0" , list.get(0));
+        assertNotNull("Mapping cache is null for platformId: 1", list.get(1));
+
+        boolean excObserved = false;
+
+        try {
+            list.get(2);
+        }
+        catch (ArrayIndexOutOfBoundsException ignored) {
+            excObserved = true;
+        }
+
+        assertTrue("ArrayIndexOutOfBoundsException had to be thrown", excObserved);
+    }
+
+    /**
+     * @param expected Expected.
+     * @param pathToReal Path to real.
+     */
+    private void checkFileName(String expected, Path pathToReal) throws IOException {
+        assertEquals(expected, new String(readAllBytes(pathToReal)));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
index 9ff127d..6085332 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/MarshallerContextTestImpl.java
@@ -21,7 +21,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.MarshallerContextAdapter;
+import org.apache.ignite.internal.MarshallerContextImpl;
 import org.apache.ignite.plugin.PluginProvider;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -29,7 +29,7 @@ import org.jsr166.ConcurrentHashMap8;
 /**
  * Test marshaller context.
  */
-public class MarshallerContextTestImpl extends MarshallerContextAdapter {
+public class MarshallerContextTestImpl extends MarshallerContextImpl {
     /** */
     private static final ConcurrentMap<Integer, String> map = new ConcurrentHashMap8<>();
 
@@ -64,29 +64,33 @@ public class MarshallerContextTestImpl extends MarshallerContextAdapter {
         this(null);
     }
 
+    /**
+     * @return Internal map.
+     */
+    public ConcurrentMap<Integer, String> internalMap() {
+        return map;
+    }
+
     /** {@inheritDoc} */
-    @Override protected boolean registerClassName(int id, String clsName) throws IgniteCheckedException {
+    @Override public boolean registerClassName(byte platformId, int typeId, String clsName) throws IgniteCheckedException {
         if (excluded != null && excluded.contains(clsName))
             return false;
 
-        String oldClsName = map.putIfAbsent(id, clsName);
+        String oldClsName = map.putIfAbsent(typeId, clsName);
 
         if (oldClsName != null && !oldClsName.equals(clsName))
-            throw new IgniteCheckedException("Duplicate ID [id=" + id + ", oldClsName=" + oldClsName + ", clsName=" +
-                clsName + ']');
+            throw new IgniteCheckedException("Duplicate ID [id=" + typeId + ", oldClsName=" + oldClsName + ", clsName=" +
+                    clsName + ']');
 
         return true;
     }
 
     /** {@inheritDoc} */
-    @Override protected String className(int id) {
-        return map.get(id);
-    }
-
-    /**
-     * @return Internal map.
-     */
-    public ConcurrentMap<Integer, String> internalMap() {
-        return map;
+    @Override public String getClassName(
+            byte platformId,
+            int typeId
+    ) throws ClassNotFoundException, IgniteCheckedException {
+        String clsName = map.get(typeId);
+        return (clsName == null) ? super.getClassName(platformId, typeId) : clsName;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerEnumSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerEnumSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerEnumSelfTest.java
index e06bed9..c072170 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerEnumSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerEnumSelfTest.java
@@ -18,19 +18,33 @@
 package org.apache.ignite.marshaller.optimized;
 
 import junit.framework.TestCase;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.marshaller.MarshallerContextTestImpl;
+import org.apache.ignite.testframework.junits.GridTestKernalContext;
+import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
 
 /**
  *
  */
 public class OptimizedMarshallerEnumSelfTest extends TestCase {
+
+    private String igniteHome = System.getProperty("user.dir");
+
+    private final IgniteLogger rootLog = new GridTestLog4jLogger(false);
     /**
      * @throws Exception If failed.
      */
     public void testEnumSerialisation() throws Exception {
         OptimizedMarshaller marsh = new OptimizedMarshaller();
 
-        marsh.setContext(new MarshallerContextTestImpl());
+        MarshallerContextTestImpl context = new MarshallerContextTestImpl();
+
+        context.onMarshallerProcessorStarted(newContext(), null);
+
+        marsh.setContext(context);
 
         byte[] bytes = marsh.marshal(TestEnum.Bond);
 
@@ -40,6 +54,15 @@ public class OptimizedMarshallerEnumSelfTest extends TestCase {
         assertEquals(TestEnum.Bond.desc, unmarshalled.desc);
     }
 
+    private GridKernalContext newContext() throws IgniteCheckedException {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setIgniteHome(igniteHome);
+        cfg.setClientMode(false);
+
+        return new GridTestKernalContext(rootLog.getLogger(OptimizedMarshallerEnumSelfTest.class), cfg);
+    }
+
     private enum TestEnum {
         Equity("Equity") {
             @Override public String getTestString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
index 1341e25..137eda2 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
@@ -23,7 +23,6 @@ import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
@@ -404,11 +403,11 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                 });
 
                 spi.setDataExchange(new DiscoverySpiDataExchange() {
-                    @Override public Map<Integer, Serializable> collect(UUID nodeId) {
-                        return new HashMap<>();
+                    @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
+                        return dataBag;
                     }
 
-                    @Override public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data) {
+                    @Override public void onExchange(DiscoveryDataBag dataBag) {
                         // No-op.
                     }
                 });

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 043208c..1ec8ee8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -68,7 +68,10 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
+import org.apache.ignite.spi.discovery.tcp.internal.DiscoveryDataPacket;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryStatistics;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder;
@@ -94,6 +97,8 @@ import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.events.EventType.EVT_NODE_METRICS_UPDATED;
 import static org.apache.ignite.events.EventType.EVT_TASK_FAILED;
 import static org.apache.ignite.events.EventType.EVT_TASK_FINISHED;
+import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.MARSHALLER_PROC;
+import static org.apache.ignite.internal.MarshallerPlatformIds.JAVA_ID;
 import static org.apache.ignite.spi.IgnitePortProtocol.UDP;
 
 /**
@@ -1972,6 +1977,39 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test verifies Ignite nodes don't exchange system types on discovery phase but only user types.
+     */
+    public void testSystemMarshallerTypesFilteredOut() throws Exception {
+        try {
+            nodeSpi.set(new TestTcpDiscoveryMarshallerDataSpi());
+
+            Ignite srv1 = startGrid(0);
+
+            IgniteCache<Object, Object> organizations = srv1.createCache("organizations");
+
+            organizations.put(1, new Organization());
+
+            startGrid(1);
+
+            assertEquals("Expected items in marshaller discovery data: 1, actual: "
+                    + TestTcpDiscoveryMarshallerDataSpi.marshalledItems,
+                    1, TestTcpDiscoveryMarshallerDataSpi.marshalledItems);
+
+            IgniteCache<Object, Object> employees = srv1.createCache("employees");
+
+            employees.put(1, new Employee());
+
+            startGrid(2);
+
+            assertEquals("Expected items in marshaller discovery data: 2, actual: "
+                    + TestTcpDiscoveryMarshallerDataSpi.marshalledItems,
+                    2, TestTcpDiscoveryMarshallerDataSpi.marshalledItems);
+        } finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testDuplicatedDiscoveryDataRemoved() throws Exception {
@@ -2084,6 +2122,51 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * SPI used in {@link #testSystemMarshallerTypesFilteredOut()} test to check that only
+     * user types get to discovery messages on joining new nodes.
+     */
+    private static class TestTcpDiscoveryMarshallerDataSpi extends TcpDiscoverySpi {
+        /** Marshalled items. */
+        static volatile int marshalledItems;
+
+        /** {@inheritDoc} */
+        @Override public TcpDiscoverySpi setDataExchange(final DiscoverySpiDataExchange exchange) {
+            return super.setDataExchange(new DiscoverySpiDataExchange() {
+                @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
+                    DiscoveryDataBag bag = exchange.collect(dataBag);
+
+                    if (bag.commonData().containsKey(MARSHALLER_PROC.ordinal()))
+                        marshalledItems = getJavaMappings(getAllMappings(dataBag)).size();
+
+                    return bag;
+                }
+
+                @Override public void onExchange(DiscoveryDataBag dataBag) {
+                    exchange.onExchange(dataBag);
+                }
+
+                private List getAllMappings(DiscoveryDataBag bag) {
+                    return (List) bag.commonData().get(MARSHALLER_PROC.ordinal());
+                }
+
+                private Map getJavaMappings(List allMappings) {
+                    return (Map) allMappings.get(JAVA_ID);
+                }
+            });
+        }
+    }
+
+    /**
+     * User class used in {@link #testSystemMarshallerTypesFilteredOut()} test to feed into marshaller cache.
+     */
+    private static class Organization { }
+
+    /**
+     * User class used in {@link #testSystemMarshallerTypesFilteredOut()} test to feed into marshaller cache.
+     */
+    private static class Employee { }
+
+    /**
      *
      */
     private static class TestDiscoveryDataDuplicateSpi extends TcpDiscoverySpi {
@@ -2101,14 +2184,22 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
             TcpDiscoveryAbstractMessage msg,
             long timeout) throws IOException, IgniteCheckedException {
             if (msg instanceof TcpDiscoveryNodeAddedMessage) {
-                Map<UUID, Map<Integer, byte[]>> discoData = ((TcpDiscoveryNodeAddedMessage)msg).oldNodesDiscoveryData();
+                DiscoveryDataPacket dataPacket = ((TcpDiscoveryNodeAddedMessage)msg).gridDiscoveryData();
 
-                checkDiscoData(discoData, msg);
+                if (dataPacket != null) {
+                    Map<UUID, Map<Integer, byte[]>> discoData = U.field(dataPacket, "nodeSpecificData");
+
+                    checkDiscoData(discoData, msg);
+                }
             }
             else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage) {
-                Map<UUID, Map<Integer, byte[]>> discoData = ((TcpDiscoveryNodeAddFinishedMessage)msg).clientDiscoData();
+                DiscoveryDataPacket dataPacket = ((TcpDiscoveryNodeAddFinishedMessage)msg).clientDiscoData();
+
+                if (dataPacket != null) {
+                    Map<UUID, Map<Integer, byte[]>> discoData = U.field(dataPacket, "nodeSpecificData");
 
-                checkDiscoData(discoData, msg);
+                    checkDiscoData(discoData, msg);
+                }
             }
 
             super.writeToSocket(sock, out, msg, timeout);
@@ -2122,9 +2213,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
             if (discoData != null && discoData.size() > 1) {
                 int cnt = 0;
 
-                for (Map.Entry<UUID, Map<Integer, byte[]>> e : discoData.entrySet()) {
-                    Map<Integer, byte[]> map = e.getValue();
-
+                for (Map<Integer, byte[]> map : discoData.values()) {
                     if (map.containsKey(GridComponent.DiscoveryDataExchangeType.CACHE_PROC.ordinal()))
                         cnt++;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiStartStopSelfTest.java
index 2a77e76..1bcc1cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiStartStopSelfTest.java
@@ -17,11 +17,8 @@
 
 package org.apache.ignite.spi.discovery.tcp;
 
-import java.io.Serializable;
-import java.util.Collections;
-import java.util.Map;
-import java.util.UUID;
 import org.apache.ignite.spi.GridSpiStartStopAbstractTest;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -47,11 +44,11 @@ public class TcpDiscoverySpiStartStopSelfTest extends GridSpiStartStopAbstractTe
     @GridSpiTestConfig
     public DiscoverySpiDataExchange getDataExchange() {
         return new DiscoverySpiDataExchange() {
-            @Override public Map<Integer, Serializable> collect(UUID nodeId) {
-                return Collections.emptyMap();
+            @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
+                return dataBag;
             }
 
-            @Override public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data) {
+            @Override public void onExchange(DiscoveryDataBag dataBag) {
                 // No-op.
             }
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 30c7244..f00298f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -79,6 +79,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoverySpiCustomMessage;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -460,7 +461,17 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Test kernal context.
      */
     protected GridTestKernalContext newContext() throws IgniteCheckedException {
-        return new GridTestKernalContext(log());
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setClientMode(false);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi() {
+            @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+                //No-op
+            }
+        });
+
+        GridTestKernalContext ctx = new GridTestKernalContext(log(), cfg);
+        return ctx;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 40f0e43..db45e27 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -64,7 +64,6 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
-                null,
                 U.allPluginProviders()
         );
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java
index 20b3cf2..c74117c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/spi/GridSpiAbstractTest.java
@@ -39,6 +39,7 @@ import org.apache.ignite.plugin.security.SecurityPermissionSet;
 import org.apache.ignite.spi.IgniteSpi;
 import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryMetricsProvider;
 import org.apache.ignite.spi.discovery.DiscoverySpi;
 import org.apache.ignite.spi.discovery.DiscoverySpiDataExchange;
@@ -220,11 +221,13 @@ public abstract class GridSpiAbstractTest<T extends IgniteSpi> extends GridAbstr
             discoSpi.setMetricsProvider(createMetricsProvider());
 
             discoSpi.setDataExchange(new DiscoverySpiDataExchange() {
-                @Override public Map<Integer, Serializable> collect(UUID nodeId) {
-                    return new HashMap<>();
+
+                @Override public DiscoveryDataBag collect(DiscoveryDataBag dataBag) {
+                    return dataBag;
                 }
 
-                @Override public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data) {
+                @Override public void onExchange(DiscoveryDataBag dataBag) {
+                    // No-op.
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 9e20d2a..d7a4a69 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -46,8 +46,8 @@ import org.apache.ignite.internal.processors.closure.GridClosureProcessorSelfTes
 import org.apache.ignite.internal.processors.closure.GridClosureSerializationTest;
 import org.apache.ignite.internal.processors.continuous.GridEventConsumeSelfTest;
 import org.apache.ignite.internal.processors.continuous.GridMessageListenSelfTest;
-import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
+import org.apache.ignite.internal.processors.odbc.OdbcProcessorValidationSelfTest;
 import org.apache.ignite.internal.processors.service.ClosureServiceClientsNodesTest;
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
 import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4cd332b7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
index 2e3d8b5..438132c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBinaryBasicTestSuite.java
@@ -30,6 +30,8 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingNe
 import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingRebalanceErrorTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingTxErrorTest;
 import org.apache.ignite.internal.processors.cache.IgniteDaemonNodeMarshallerCacheTest;
+import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClassNameConflictTest;
+import org.apache.ignite.internal.processors.cache.IgniteMarshallerCacheClientRequestsMappingOnMissTest;
 import org.apache.ignite.internal.util.GridHandleTableSelfTest;
 import org.apache.ignite.internal.util.IgniteUtilsSelfTest;
 import org.apache.ignite.internal.util.io.GridUnsafeDataOutputArraySizingSelfTest;
@@ -94,6 +96,9 @@ public class IgniteBinaryBasicTestSuite extends TestSuite {
 
         suite.addTest(IgniteBasicTestSuite.suite(ignoredTests));
 
+        suite.addTestSuite(IgniteMarshallerCacheClassNameConflictTest.class);
+        suite.addTestSuite(IgniteMarshallerCacheClientRequestsMappingOnMissTest.class);
+
         return suite;
     }
 }