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

[22/27] ignite git commit: IGNITE-5259 Minor serialization fix

IGNITE-5259 Minor serialization fix


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

Branch: refs/heads/ignite-5232-1.7.2
Commit: b2040b7a95e421609bcf7ae05b56dc623310b409
Parents: 04fadd4
Author: dkarachentsev <dk...@gridgain.com>
Authored: Tue May 23 16:14:08 2017 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Tue May 23 16:14:08 2017 +0300

----------------------------------------------------------------------
 modules/core/pom.xml                            |  3 +
 .../apache/ignite/IgniteSystemProperties.java   | 12 ++-
 .../ignite/internal/IgniteNodeAttributes.java   |  6 ++
 .../discovery/GridDiscoveryManager.java         | 39 ++++++++
 .../continuous/CacheContinuousQueryHandler.java |  1 -
 .../top/GridTopologyCommandHandler.java         |  2 +
 .../processors/security/SecurityUtils.java      | 92 +++++++++++++++++++
 .../security/SecurityBasicPermissionSet.java    | 41 ++++++++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 97 ++++++++++++++++++--
 .../GridDiscoveryManagerAttributesSelfTest.java | 70 +++++++++++++-
 .../discovery/tcp/TestReconnectProcessor.java   | 47 +++++++++-
 11 files changed, 390 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 4c4343a..4a14407 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -234,6 +234,9 @@
                     <exclude>**/*.java</exclude>
                 </excludes>
             </testResource>
+            <testResource>
+                <directory>src/test/resources</directory>
+            </testResource>
         </testResources>
 
         <plugins>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 713defe..6827e0c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -503,7 +503,7 @@ public final class IgniteSystemProperties {
     /**
      * Whether Ignite can access unaligned memory addresses.
      * <p>
-     * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture.
+     * Defaults to {@code false}, meaning that unaligned access will be performed only on x86 architecture.
      */
     public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS";
 
@@ -523,6 +523,16 @@ public final class IgniteSystemProperties {
      */
     public static final String IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE = "IGNITE_USE_LOCAL_BINARY_MARSHALLER_CACHE";
 
+     /**
+     * When set to {@code true}, Ignite switches to compatibility mode with versions that don't
+     * support service security permissions. In this case security permissions will be ignored
+     * (if they set).
+     * <p>
+     *     Default is {@code false}, which means that service security permissions will be respected.
+     * </p>
+     */
+    public static final String IGNITE_SECURITY_COMPATIBILITY_MODE = "IGNITE_SECURITY_COMPATIBILITY_MODE";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index e7c984f..4367924 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -136,6 +136,9 @@ public final class IgniteNodeAttributes {
     /** Security subject for authenticated node. */
     public static final String ATTR_SECURITY_SUBJECT = ATTR_PREFIX + ".security.subject";
 
+    /** V2 security subject for authenticated node. */
+    public static final String ATTR_SECURITY_SUBJECT_V2 = ATTR_PREFIX + ".security.subject.v2";
+
     /** Client mode flag. */
     public static final String ATTR_CLIENT_MODE = ATTR_PREFIX + ".cache.client";
 
@@ -160,6 +163,9 @@ public final class IgniteNodeAttributes {
     /** Ignite services compatibility mode (can be {@code null}). */
     public static final String ATTR_SERVICES_COMPATIBILITY_MODE = ATTR_PREFIX + ".services.compatibility.enabled";
 
+    /** Ignite security compatibility mode. */
+    public static final String ATTR_SECURITY_COMPATIBILITY_MODE = ATTR_PREFIX + ".security.compatibility.enabled";
+
     /**
      * Enforces singleton.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/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 2ec1070..b3ba83d 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
@@ -121,6 +121,7 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISCOVERY_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.events.EventType.EVT_CLIENT_NODE_DISCONNECTED;
@@ -136,9 +137,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_COMPATIBILITY_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_USER_NAME;
 import static org.apache.ignite.internal.IgniteVersionUtils.VER;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.SERVICE_PERMISSIONS_SINCE;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode;
 import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.NOOP;
 
 /**
@@ -442,6 +446,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         spi.setMetricsProvider(createMetricsProvider());
 
         if (ctx.security().enabled()) {
+            if (isSecurityCompatibilityMode())
+                ctx.addNodeAttribute(ATTR_SECURITY_COMPATIBILITY_MODE, true);
+
             spi.setAuthenticator(new DiscoverySpiNodeAuthenticator() {
                 @Override public SecurityContext authenticateNode(ClusterNode node, SecurityCredentials cred) {
                     try {
@@ -1044,6 +1051,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         boolean locDelayAssign = locNode.attribute(ATTR_LATE_AFFINITY_ASSIGNMENT);
 
         Boolean locSrvcCompatibilityEnabled = locNode.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
+        Boolean locSecurityCompatibilityEnabled = locNode.attribute(ATTR_SECURITY_COMPATIBILITY_MODE);
 
         for (ClusterNode n : nodes) {
             int rmtJvmMajVer = nodeJavaMajorVersion(n);
@@ -1157,6 +1165,37 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     ", rmtNodeAddrs=" + U.addressesAsString(n) +
                     ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
             }
+
+            if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) >= 0
+                && ctx.security().enabled() // Matters only if security enabled.
+               ) {
+                Boolean rmtSecurityCompatibilityEnabled = n.attribute(ATTR_SECURITY_COMPATIBILITY_MODE);
+
+                if (!F.eq(locSecurityCompatibilityEnabled, rmtSecurityCompatibilityEnabled)) {
+                    throw new IgniteCheckedException("Local node's " + IGNITE_SECURITY_COMPATIBILITY_MODE +
+                        " property value differs from remote node's value " +
+                        "(to make sure all nodes in topology have identical Ignite security compatibility mode enabled, " +
+                        "configure system property explicitly) " +
+                        "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled +
+                        ", rmtSecurityCompatibilityEnabled=" + rmtSecurityCompatibilityEnabled +
+                        ", locNodeAddrs=" + U.addressesAsString(locNode) +
+                        ", rmtNodeAddrs=" + U.addressesAsString(n) +
+                        ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ']');
+                }
+            }
+
+            if (n.version().compareToIgnoreTimestamp(SERVICE_PERMISSIONS_SINCE) < 0
+                && ctx.security().enabled() // Matters only if security enabled.
+                && (locSecurityCompatibilityEnabled == null || !locSecurityCompatibilityEnabled)) {
+                throw new IgniteCheckedException("Remote node does not support service security permissions. " +
+                    "To be able to join to it, local node must be started with " + IGNITE_SECURITY_COMPATIBILITY_MODE +
+                    " system property set to \"true\". " +
+                    "[locSecurityCompatibilityEnabled=" + locSecurityCompatibilityEnabled +
+                    ", locNodeAddrs=" + U.addressesAsString(locNode) +
+                    ", rmtNodeAddrs=" + U.addressesAsString(n) +
+                    ", locNodeId=" + locNode.id() + ", rmtNodeId=" + n.id() + ", " +
+                    ", rmtNodeVer" + n.version() + ']');
+            }
         }
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index 5f00d58..17f4308 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -998,7 +998,6 @@ public class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler
 
             synchronized (pendingEvts) {
                 if (log.isDebugEnabled()) {
-
                     log.debug("Handling event [lastFiredEvt=" + lastFiredEvt +
                         ", curTop=" + curTop +
                         ", entUpdCnt=" + entry.updateCounter() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index 297785e..3c68fbf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -58,6 +58,7 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_HOST
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_REST_TCP_PORT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_TX_CONFIG;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.NODE;
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.TOPOLOGY;
@@ -290,6 +291,7 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
             attrs.remove(ATTR_CACHE);
             attrs.remove(ATTR_TX_CONFIG);
             attrs.remove(ATTR_SECURITY_SUBJECT);
+            attrs.remove(ATTR_SECURITY_SUBJECT_V2);
             attrs.remove(ATTR_SECURITY_CREDENTIALS);
 
             for (Iterator<Map.Entry<String, Object>> i = attrs.entrySet().iterator(); i.hasNext();) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
new file mode 100644
index 0000000..1016335
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityUtils.java
@@ -0,0 +1,92 @@
+/*
+ * 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.security;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.apache.ignite.plugin.security.SecurityPermission;
+
+/**
+ * Security utilities.
+ */
+public class SecurityUtils {
+    /** Version since service security supported. */
+    public static final IgniteProductVersion SERVICE_PERMISSIONS_SINCE = IgniteProductVersion.fromString("1.7.11");
+
+    /** Default serialization version. */
+    private final static int DFLT_SERIALIZE_VERSION = isSecurityCompatibilityMode() ? 1 : 2;
+
+    /** Current serialization version. */
+    private static final ThreadLocal<Integer> SERIALIZE_VERSION = new ThreadLocal<Integer>(){
+        @Override protected Integer initialValue() {
+            return DFLT_SERIALIZE_VERSION;
+        }
+    };
+
+    /**
+     * Private constructor.
+     */
+    private SecurityUtils() {
+    }
+
+    /**
+     * @return Security compatibility mode flag.
+     */
+    public static boolean isSecurityCompatibilityMode() {
+        return IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE, false);
+    }
+
+    /**
+     * @param ver Serialize version.
+     */
+    public static void serializeVersion(int ver) {
+        SERIALIZE_VERSION.set(ver);
+    }
+
+    /**
+     * @return Serialize version.
+     */
+    public static int serializeVersion() {
+        return SERIALIZE_VERSION.get();
+    }
+
+    /**
+     * Sets default serialize version {@link #DFLT_SERIALIZE_VERSION}.
+     */
+    public static void restoreDefaultSerializeVersion() {
+        serializeVersion(DFLT_SERIALIZE_VERSION);
+    }
+
+    /**
+     * @return Allow all service permissions.
+     */
+    public static Map<String, Collection<SecurityPermission>> compatibleServicePermissions() {
+        Map<String, Collection<SecurityPermission>> srvcPerms = new HashMap<>();
+
+        srvcPerms.put("*", Arrays.asList(
+            SecurityPermission.SERVICE_CANCEL,
+            SecurityPermission.SERVICE_DEPLOY,
+            SecurityPermission.SERVICE_INVOKE));
+
+        return srvcPerms;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java
index 44166d9..370eadd 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityBasicPermissionSet.java
@@ -17,15 +17,24 @@
 
 package org.apache.ignite.plugin.security;
 
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.internal.processors.security.SecurityUtils.compatibleServicePermissions;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.isSecurityCompatibilityMode;
+import static org.apache.ignite.internal.processors.security.SecurityUtils.serializeVersion;
+
 /**
  * Simple implementation of {@link SecurityPermissionSet} interface. Provides
  * convenient way to specify permission set in the XML configuration.
@@ -44,7 +53,9 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet {
 
     /** Service permissions. */
     @GridToStringInclude
-    private Map<String, Collection<SecurityPermission>> servicePermissions = new HashMap<>();
+    private transient Map<String, Collection<SecurityPermission>> servicePermissions = isSecurityCompatibilityMode()
+            ? compatibleServicePermissions()
+            : new HashMap<String, Collection<SecurityPermission>>();
 
     /** System permissions. */
     @GridToStringInclude
@@ -158,6 +169,34 @@ public class SecurityBasicPermissionSet implements SecurityPermissionSet {
         return res;
     }
 
+    /**
+     * @param out Out.
+     */
+    private void writeObject(ObjectOutputStream out) throws IOException {
+        out.defaultWriteObject();
+
+        if (serializeVersion() >= 2)
+            U.writeMap(out, servicePermissions);
+    }
+
+    /**
+     * @param in In.
+     */
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+        in.defaultReadObject();
+
+        if (serializeVersion() >= 2)
+            servicePermissions = U.readMap(in);
+
+        if (servicePermissions == null) {
+            // Allow all for compatibility mode
+            if (serializeVersion() < 2)
+                servicePermissions = compatibleServicePermissions();
+            else
+                servicePermissions = Collections.emptyMap();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(SecurityBasicPermissionSet.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/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 afd1c2b..58b362f 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
@@ -72,6 +72,7 @@ import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager;
 import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.processors.security.SecurityUtils;
 import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.util.GridBoundedLinkedHashSet;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
@@ -947,7 +948,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             Map<String, Object> attrs = new HashMap<>(locNode.attributes());
 
-            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj));
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj));
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1));
 
             locNode.setAttributes(attrs);
 
@@ -983,7 +985,16 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             for (InetSocketAddress addr : addrs) {
                 try {
-                    Integer res = sendMessageDirectly(joinReq, addr);
+                    Integer res;
+
+                    try {
+                        SecurityUtils.serializeVersion(1);
+
+                        res = sendMessageDirectly(joinReq, addr);
+                    }
+                    finally {
+                        SecurityUtils.restoreDefaultSerializeVersion();
+                    }
 
                     assert res != null;
 
@@ -1922,6 +1933,39 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /**
+     * @param obj Object.
+     * @param ver Security serialize version.
+     * @return Marshaled object.
+     */
+    private byte[] marshalWithSecurityVersion(Object obj, int ver) throws IgniteCheckedException {
+        try {
+            SecurityUtils.serializeVersion(ver);
+
+            return U.marshal(spi.marshaller(), obj);
+        }
+        finally {
+            SecurityUtils.restoreDefaultSerializeVersion();
+        }
+    }
+
+    /**
+     * @param bytes Marshaled object.
+     * @param ver Security serialize version.
+     * @return Unmarshaled object.
+     */
+    private <T> T unmarshalWithSecurityVersion(byte[] bytes, int ver) throws IgniteCheckedException {
+        try {
+            if (ver > 0)
+                SecurityUtils.serializeVersion(ver);
+
+            return spi.marshaller().unmarshal(bytes, U.resolveClassLoader(spi.ignite().configuration()));
+        }
+        finally {
+            SecurityUtils.restoreDefaultSerializeVersion();
+        }
+    }
+
+    /**
      * Discovery messages history used for client reconnect.
      */
     private class EnsuredMessageHistory {
@@ -2995,6 +3039,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     pendingMsgs.customDiscardId);
 
                             try {
+                                SecurityUtils.serializeVersion(1);
+
                                 long tstamp = U.currentTimeMillis();
 
                                 if (timeoutHelper == null)
@@ -3033,6 +3079,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                                 }
                             }
                             finally {
+                                SecurityUtils.restoreDefaultSerializeVersion();
+
                                 clearNodeAddedMessage(msg);
                             }
 
@@ -3423,7 +3471,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                             // Stick in authentication subject to node (use security-safe attributes for copy).
                             Map<String, Object> attrs = new HashMap<>(node.getAttributes());
 
-                            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, U.marshal(spi.marshaller(), subj));
+                            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2, U.marshal(spi.marshaller(), subj));
+                            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, marshalWithSecurityVersion(subj, 1));
 
                             node.setAttributes(attrs);
                         }
@@ -4073,9 +4122,22 @@ class ServerImpl extends TcpDiscoveryImpl {
                         else {
                             SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
-                            SecurityContext coordSubj = U.unmarshal(spi.marshaller(),
-                                node.<byte[]>attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT),
-                                U.resolveClassLoader(spi.ignite().configuration()));
+                            byte[] subjBytes = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT);
+                            byte[] subjBytesV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2);
+
+                            SecurityContext coordSubj;
+
+                            try {
+                                if (subjBytesV2 == null)
+                                    SecurityUtils.serializeVersion(1);
+
+                                coordSubj = U.unmarshal(spi.marshaller(),
+                                    subjBytesV2 != null ? subjBytesV2 : subjBytes,
+                                    U.resolveClassLoader(spi.ignite().configuration()));
+                            }
+                            finally {
+                                SecurityUtils.restoreDefaultSerializeVersion();
+                            }
 
                             if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) {
                                 // Node has not pass authentication.
@@ -4158,13 +4220,23 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     new TcpDiscoveryAuthFailedMessage(locNodeId, spi.locHost);
 
                                 try {
-                                    ClassLoader cl = U.resolveClassLoader(spi.ignite().configuration());
-
                                     byte[] rmSubj = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT);
                                     byte[] locSubj = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT);
 
-                                    SecurityContext rmCrd = spi.marshaller().unmarshal(rmSubj, cl);
-                                    SecurityContext locCrd = spi.marshaller().unmarshal(locSubj, cl);
+                                    byte[] rmSubjV2 = node.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2);
+                                    byte[] locSubjV2 = locNode.attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT_V2);
+
+                                    int ver = 1; // Compatible version.
+
+                                    if (rmSubjV2 != null && locSubjV2 != null) {
+                                        rmSubj = rmSubjV2;
+                                        locSubj = locSubjV2;
+
+                                        ver = 0; // Default version.
+                                    }
+
+                                    SecurityContext rmCrd = unmarshalWithSecurityVersion(rmSubj, ver);
+                                    SecurityContext locCrd = unmarshalWithSecurityVersion(locSubj, ver);
 
                                     if (!permissionsEqual(locCrd.subject().permissions(),
                                         rmCrd.subject().permissions())) {
@@ -5812,6 +5884,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 while (!isInterrupted()) {
                     try {
+                        SecurityUtils.serializeVersion(1);
+
                         TcpDiscoveryAbstractMessage msg = U.unmarshal(spi.marshaller(), in,
                             U.resolveClassLoader(spi.ignite().configuration()));
 
@@ -6062,6 +6136,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                         return;
                     }
+                    finally {
+                        SecurityUtils.restoreDefaultSerializeVersion();
+                    }
                 }
             }
             finally {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index ba8fa5b..f0096db 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -21,14 +21,18 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TestReconnectPluginProvider;
+import org.apache.ignite.spi.discovery.tcp.TestReconnectProcessor;
 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.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_SECURITY_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SERVICES_COMPATIBILITY_MODE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS;
@@ -258,18 +262,69 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
      * @throws Exception If failed.
      */
     private void doTestServiceCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception {
+        doTestCompatibilityEnabled(IGNITE_SERVICES_COMPATIBILITY_MODE, first, second, fail);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSecurityCompatibilityEnabled() throws Exception {
+        TestReconnectPluginProvider.enabled = true;
+        TestReconnectProcessor.enabled = true;
+
+        try {
+            doTestSecurityCompatibilityEnabled(true, null, true);
+            doTestSecurityCompatibilityEnabled(true, false, true);
+            doTestSecurityCompatibilityEnabled(false, true, true);
+            doTestSecurityCompatibilityEnabled(null, true, true);
+
+            doTestSecurityCompatibilityEnabled(null, null, false);
+            doTestSecurityCompatibilityEnabled(null, false, false);
+            doTestSecurityCompatibilityEnabled(false, false, false);
+            doTestSecurityCompatibilityEnabled(false, null, false);
+            doTestSecurityCompatibilityEnabled(true, true, false);
+        }
+        finally {
+            TestReconnectPluginProvider.enabled = false;
+            TestReconnectProcessor.enabled = false;
+        }
+    }
+
+    /**
+     * @param first Service compatibility enabled flag for first node.
+     * @param second Service compatibility enabled flag for second node.
+     * @param fail Fail flag.
+     * @throws Exception If failed.
+     */
+    private void doTestSecurityCompatibilityEnabled(Object first, Object second, boolean fail) throws Exception {
+        doTestCompatibilityEnabled(IGNITE_SECURITY_COMPATIBILITY_MODE, first, second, fail);
+    }
+
+    /**
+     * @param prop System property.
+     * @param first Service compatibility enabled flag for first node.
+     * @param second Service compatibility enabled flag for second node.
+     * @param fail Fail flag.
+     * @throws Exception If failed.
+     */
+    private void doTestCompatibilityEnabled(String prop, Object first, Object second, boolean fail) throws Exception {
+        String backup = System.getProperty(prop);
         try {
             if (first != null)
-                System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(first));
+                System.setProperty(prop, String.valueOf(first));
             else
-                System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE);
+                System.clearProperty(prop);
 
-            startGrid(0);
+            IgniteEx ignite = startGrid(0);
+
+            // Ignore if disabled security plugin used.
+            if (IGNITE_SECURITY_COMPATIBILITY_MODE.equals(prop) && !ignite.context().security().enabled())
+                return;
 
             if (second != null)
-                System.setProperty(IGNITE_SERVICES_COMPATIBILITY_MODE, String.valueOf(second));
+                System.setProperty(prop, String.valueOf(second));
             else
-                System.clearProperty(IGNITE_SERVICES_COMPATIBILITY_MODE);
+                System.clearProperty(prop);
 
             try {
                 startGrid(1);
@@ -284,6 +339,11 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
         }
         finally {
             stopAllGrids();
+
+            if (backup != null)
+                System.setProperty(prop, backup);
+            else
+                System.clearProperty(prop);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2040b7a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java
index f0ed35c..2476bd3 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TestReconnectProcessor.java
@@ -17,11 +17,13 @@
 
 package org.apache.ignite.spi.discovery.tcp;
 
+import java.io.Serializable;
 import java.util.Collection;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
 import org.apache.ignite.internal.processors.security.GridSecurityProcessor;
 import org.apache.ignite.internal.processors.security.SecurityContext;
@@ -37,6 +39,9 @@ import org.jetbrains.annotations.Nullable;
  * Updates node attributes on disconnect.
  */
 public class TestReconnectProcessor extends GridProcessorAdapter implements GridSecurityProcessor {
+    /** Enabled flag. */
+    public static boolean enabled;
+
     /**
      * @param ctx Kernal context.
      */
@@ -45,9 +50,14 @@ public class TestReconnectProcessor extends GridProcessorAdapter implements Grid
     }
 
     /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        ctx.addNodeAttribute(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS, new SecurityCredentials());
+    }
+
+    /** {@inheritDoc} */
     @Override public SecurityContext authenticateNode(ClusterNode node,
         SecurityCredentials cred) throws IgniteCheckedException {
-        return null;
+        return new TestSecurityContext();
     }
 
     /** {@inheritDoc} */
@@ -83,11 +93,44 @@ public class TestReconnectProcessor extends GridProcessorAdapter implements Grid
 
     /** {@inheritDoc} */
     @Override public boolean enabled() {
-        return false;
+        return enabled;
     }
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
         ctx.addNodeAttribute("test", "2");
     }
+
+    /**
+     *
+     */
+    private static class TestSecurityContext implements SecurityContext, Serializable {
+        /** Serial version uid. */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public SecuritySubject subject() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean taskOperationAllowed(String taskClsName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean cacheOperationAllowed(String cacheName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean serviceOperationAllowed(String srvcName, SecurityPermission perm) {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean systemOperationAllowed(SecurityPermission perm) {
+            return true;
+        }
+    }
 }