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/03/29 14:02:48 UTC

[24/50] [abbrv] ignite git commit: IGNITE-4827: Remove compatibility logic for 1.x versions. This closes #1654.

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index 1f21fd2..9805a66 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.internal.processors.closure;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -70,7 +66,6 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.internal.util.worker.GridWorkerFuture;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteClosure;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.resources.LoadBalancerResource;
@@ -87,9 +82,6 @@ import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKe
  *
  */
 public class GridClosureProcessor extends GridProcessorAdapter {
-    /** Ignite version in which binarylizable versions of closures were introduced. */
-    public static final IgniteProductVersion BINARYLIZABLE_CLOSURES_SINCE = IgniteProductVersion.fromString("1.6.0");
-
     /** Pool processor. */
     private final PoolProcessor pools;
 
@@ -262,7 +254,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     case BROADCAST: {
                         for (ClusterNode n : nodes)
                             for (Runnable r : jobs)
-                                mapper.map(downgradeJobIfNeeded(job(r), n), n);
+                                mapper.map(job(r), n);
 
                         break;
                     }
@@ -273,7 +265,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
                             ClusterNode n = lb.getBalancedNode(job, null);
 
-                            mapper.map(downgradeJobIfNeeded(job, n), n);
+                            mapper.map(job, n);
                         }
 
                         break;
@@ -316,7 +308,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     case BROADCAST: {
                         for (ClusterNode n : nodes)
                             for (Callable<R> c : jobs)
-                                mapper.map(downgradeJobIfNeeded(job(c), n), n);
+                                mapper.map(job(c), n);
 
                         break;
                     }
@@ -327,7 +319,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
                             ClusterNode n = lb.getBalancedNode(job, null);
 
-                            mapper.map(downgradeJobIfNeeded(job, n), n);
+                            mapper.map(job, n);
                         }
 
                         break;
@@ -1018,7 +1010,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static <T, R> ComputeJob job(final IgniteClosure<T, R> job, @Nullable final T arg) {
         A.notNull(job, "job");
 
-        return job instanceof ComputeJobMasterLeaveAware ? new C1MLAV2<>(job, arg) : new C1V2<>(job, arg);
+        return job instanceof ComputeJobMasterLeaveAware ? new C1MLA<>(job, arg) : new C1<>(job, arg);
     }
 
     /**
@@ -1030,7 +1022,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static <R> ComputeJob job(final Callable<R> c) {
         A.notNull(c, "job");
 
-        return c instanceof ComputeJobMasterLeaveAware ? new C2MLAV2<>(c) : new C2V2<>(c);
+        return c instanceof ComputeJobMasterLeaveAware ? new C2MLA<>(c) : new C2<>(c);
     }
 
     /**
@@ -1042,46 +1034,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     private static ComputeJob job(final Runnable r) {
         A.notNull(r, "job");
 
-        return r instanceof ComputeJobMasterLeaveAware ? new C4MLAV2(r) : new C4V2(r);
-    }
-
-    /**
-     * Downgrades provided job to older version if target does not support it.
-     *
-     * @param job Job.
-     * @param node Node.
-     * @return Provided or downgraded job.
-     */
-    private static ComputeJob downgradeJobIfNeeded(ComputeJob job, ClusterNode node) {
-        A.notNull(job, "job");
-
-        assert node != null;
-
-        IgniteProductVersion nodeVer = node.version();
-
-        if (nodeVer.compareTo(BINARYLIZABLE_CLOSURES_SINCE) >= 0)
-            return job;
-
-        if (job instanceof C1V2) {
-            if (job instanceof C1MLAV2)
-                return new C1MLA<>(((C1MLAV2)job).job, ((C1MLAV2)job).arg);
-            else
-                return new C1<>(((C1V2)job).job, ((C1V2)job).arg);
-        }
-        else if (job instanceof C2V2) {
-            if (job instanceof C2MLAV2)
-                return new C2MLA<>(((C2MLAV2)job).c);
-            else
-                return new C2<>(((C2V2)job).c);
-        }
-        else if (job instanceof C4V2) {
-            if (job instanceof C4MLAV2)
-                return new C4MLA(((C4MLAV2)job).r);
-            else
-                return new C4(((C4V2)job).r);
-        }
-
-        return job;
+        return r instanceof ComputeJobMasterLeaveAware ? new C4MLA(r) : new C4(r);
     }
 
     /**
@@ -1113,12 +1066,6 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /** */
         private boolean hadLocNode;
 
-        /** */
-        private byte[] closureBytes;
-
-        /** */
-        private IgniteClosure<?, ?> closure;
-
         /**
          * @param expJobCnt Expected Jobs count.
          */
@@ -1136,22 +1083,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 if (hadLocNode) {
                     Marshaller marsh = ctx.config().getMarshaller();
 
-                    if (job instanceof C1) {
-                        C1 c = (C1)job;
-
-                        if (closureBytes == null) {
-                            closure = c.job;
-
-                            closureBytes = U.marshal(marsh, c.job);
-                        }
-
-                        if (c.job == closure)
-                            c.job = U.unmarshal(marsh, closureBytes, U.resolveClassLoader(ctx.config()));
-                        else
-                            c.job = U.unmarshal(marsh, U.marshal(marsh, c.job), U.resolveClassLoader(ctx.config()));
-                    }
-                    else
-                        job = U.unmarshal(marsh, U.marshal(marsh, job), U.resolveClassLoader(ctx.config()));
+                    job = U.unmarshal(marsh, U.marshal(marsh, job), U.resolveClassLoader(ctx.config()));
                 }
                 else
                     hadLocNode = true;
@@ -1351,7 +1283,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) {
-            return Collections.singletonMap(downgradeJobIfNeeded(job(this.job), node), node);
+            return Collections.singletonMap(job(job), node);
         }
 
         /** {@inheritDoc} */
@@ -1415,7 +1347,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg) {
-            return Collections.singletonMap(downgradeJobIfNeeded(job(this.job), node), node);
+            return Collections.singletonMap(job(job), node);
         }
 
         /** {@inheritDoc} */
@@ -1560,7 +1492,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
             ClusterNode node = lb.getBalancedNode(job, null);
 
-            return Collections.singletonMap(downgradeJobIfNeeded(job, node), node);
+            return Collections.singletonMap(job, node);
         }
 
         /** {@inheritDoc} */
@@ -1611,7 +1543,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
                     ClusterNode node = lb.getBalancedNode(job, null);
 
-                    mapper.map(downgradeJobIfNeeded(job, node), node);
+                    mapper.map(job, node);
                 }
 
                 return mapper.map();
@@ -1669,7 +1601,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
                     ClusterNode node = lb.getBalancedNode(job, null);
 
-                    mapper.map(downgradeJobIfNeeded(job, node), node);
+                    mapper.map(job, node);
                 }
 
                 return mapper.map();
@@ -1723,7 +1655,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 JobMapper mapper = new JobMapper(subgrid.size());
 
                 for (ClusterNode n : subgrid)
-                    mapper.map(downgradeJobIfNeeded(job(job, arg), n), n);
+                    mapper.map(job(job, arg), n);
 
                 return mapper.map();
             }
@@ -1741,7 +1673,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    private static class C1<T, R> implements ComputeJob, Externalizable, GridNoImplicitInjection,
+    public static class C1<T, R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
         GridInternalWrapper<IgniteClosure> {
         /** */
         private static final long serialVersionUID = 0L;
@@ -1752,7 +1684,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** */
         @GridToStringInclude(sensitive = true)
-        private T arg;
+        protected T arg;
 
         /**
          *
@@ -1780,71 +1712,6 @@ public class GridClosureProcessor extends GridProcessorAdapter {
             // No-op.
         }
 
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(job);
-            out.writeObject(arg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            job = (IgniteClosure<T, R>)in.readObject();
-            arg = (T)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteClosure userObject() {
-            return job;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C1.class, this);
-        }
-    }
-
-    /**
-     *
-     */
-    public static class C1V2<T, R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
-        GridInternalWrapper<IgniteClosure> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        @GridToStringInclude
-        protected IgniteClosure<T, R> job;
-
-        /** */
-        @GridToStringInclude(sensitive = true)
-        protected T arg;
-
-        /**
-         *
-         */
-        public C1V2() {
-            // No-op.
-        }
-
-        /**
-         * @param job Job.
-         * @param arg Argument.
-         */
-        C1V2(IgniteClosure<T, R> job, T arg) {
-            this.job = job;
-            this.arg = arg;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object execute() {
-            return job.apply(arg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            // No-op.
-        }
-
         @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
             BinaryRawWriter rawWriter = writer.rawWriter();
 
@@ -1866,14 +1733,14 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            return S.toString(C1V2.class, this);
+            return S.toString(C1.class, this);
         }
     }
 
     /**
      *
      */
-    private static class C1MLA<T, R> extends C1<T, R> implements ComputeJobMasterLeaveAware {
+    public static class C1MLA<T, R> extends C1<T, R> implements ComputeJobMasterLeaveAware {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1906,40 +1773,8 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     /**
      *
      */
-    public static class C1MLAV2<T, R> extends C1V2<T, R> implements ComputeJobMasterLeaveAware {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public C1MLAV2() {
-            // No-op.
-        }
-
-        /**
-         * @param job Job.
-         * @param arg Argument.
-         */
-        private C1MLAV2(IgniteClosure<T, R> job, T arg) {
-            super(job, arg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
-            ((ComputeJobMasterLeaveAware)job).onMasterNodeLeft(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C1MLAV2.class, this, super.toString());
-        }
-    }
-
-    /**
-     *
-     */
-    private static class C2<R> implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Callable> {
+    public static class C2<R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
+        GridInternalWrapper<Callable> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1976,68 +1811,6 @@ public class GridClosureProcessor extends GridProcessorAdapter {
             // No-op.
         }
 
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(c);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            c = (Callable<R>)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Callable userObject() {
-            return c;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C2.class, this);
-        }
-    }
-
-    /**
-     *
-     */
-    public static class C2V2<R> implements ComputeJob, Binarylizable, GridNoImplicitInjection,
-        GridInternalWrapper<Callable> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        @GridToStringInclude
-        protected Callable<R> c;
-
-        /**
-         *
-         */
-        public C2V2() {
-            // No-op.
-        }
-
-        /**
-         * @param c Callable.
-         */
-        private C2V2(Callable<R> c) {
-            this.c = c;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object execute() {
-            try {
-                return c.call();
-            }
-            catch (Exception e) {
-                throw new IgniteException(e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            // No-op.
-        }
-
         @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
             writer.rawWriter().writeObject(c);
         }
@@ -2053,14 +1826,14 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            return S.toString(C2V2.class, this);
+            return S.toString(C2.class, this);
         }
     }
 
     /**
      *
      */
-    private static class C2MLA<R> extends C2<R> implements ComputeJobMasterLeaveAware {
+    public static class C2MLA<R> extends C2<R> implements ComputeJobMasterLeaveAware {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2090,40 +1863,8 @@ public class GridClosureProcessor extends GridProcessorAdapter {
     }
 
     /**
-     *
      */
-    public static class C2MLAV2<R> extends C2V2<R> implements ComputeJobMasterLeaveAware {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public C2MLAV2() {
-            // No-op.
-        }
-
-        /**
-         * @param c Callable.
-         */
-        private C2MLAV2(Callable<R> c) {
-            super(c);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
-            ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C2MLAV2.class, this, super.toString());
-        }
-    }
-
-    /**
-     */
-    private static class C4 implements ComputeJob, Externalizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
+    public static class C4 implements ComputeJob, Binarylizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2157,63 +1898,6 @@ public class GridClosureProcessor extends GridProcessorAdapter {
             // No-op.
         }
 
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(r);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            r = (Runnable)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Runnable userObject() {
-            return r;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C4.class, this);
-        }
-    }
-
-    /**
-     */
-    public static class C4V2 implements ComputeJob, Binarylizable, GridNoImplicitInjection, GridInternalWrapper<Runnable> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        @GridToStringInclude
-        protected Runnable r;
-
-        /**
-         *
-         */
-        public C4V2() {
-            // No-op.
-        }
-
-        /**
-         * @param r Runnable.
-         */
-        private C4V2(Runnable r) {
-            this.r = r;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object execute() {
-            r.run();
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            // No-op.
-        }
-
         @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
             writer.rawWriter().writeObject(r);
         }
@@ -2229,14 +1913,14 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public String toString() {
-            return S.toString(C4V2.class, this);
+            return S.toString(C4.class, this);
         }
     }
 
     /**
      *
      */
-    private static class C4MLA extends C4 implements ComputeJobMasterLeaveAware {
+    public static class C4MLA extends C4 implements ComputeJobMasterLeaveAware {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2264,36 +1948,4 @@ public class GridClosureProcessor extends GridProcessorAdapter {
             return S.toString(C4MLA.class, this, super.toString());
         }
     }
-
-    /**
-     *
-     */
-    public static class C4MLAV2 extends C4V2 implements ComputeJobMasterLeaveAware {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         *
-         */
-        public C4MLAV2() {
-            // No-op.
-        }
-
-        /**
-         * @param r Runnable.
-         */
-        private C4MLAV2(Runnable r) {
-            super(r);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMasterNodeLeft(ComputeTaskSession ses) {
-            ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(C4MLAV2.class, this, super.toString());
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/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 ffdf3ba..641208d 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
@@ -117,9 +117,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /** Threads started by this processor. */
     private final Map<UUID, IgniteThread> bufCheckThreads = new ConcurrentHashMap8<>();
 
-    /**  */
-    public static final IgniteProductVersion QUERY_MSG_VER_2_SINCE = IgniteProductVersion.fromString("1.5.9");
-
     /** */
     private final ConcurrentMap<IgniteUuid, SyncMessageAckFuture> syncMsgFuts = new ConcurrentHashMap8<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
deleted file mode 100644
index 8314549..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/messages/GridQueryRequest.java
+++ /dev/null
@@ -1,368 +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.processors.query.h2.twostep.messages;
-
-import java.nio.ByteBuffer;
-import java.util.Collection;
-import java.util.List;
-import org.apache.ignite.internal.GridDirectCollection;
-import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
-import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
-import org.apache.ignite.internal.util.tostring.GridToStringInclude;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.plugin.extensions.communication.Message;
-import org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType;
-import org.apache.ignite.plugin.extensions.communication.MessageReader;
-import org.apache.ignite.plugin.extensions.communication.MessageWriter;
-
-/**
- * Query request.
- */
-@Deprecated
-@IgniteCodeGeneratingFail
-public class GridQueryRequest implements Message, GridCacheQueryMarshallable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private long reqId;
-
-    /** */
-    private int pageSize;
-
-    /** */
-    private String space;
-
-    /** */
-    @GridToStringInclude
-    @GridDirectCollection(GridCacheSqlQuery.class)
-    private Collection<GridCacheSqlQuery> qrys;
-
-    /** Topology version. */
-    private AffinityTopologyVersion topVer;
-
-    /** */
-    @GridToStringInclude
-    @GridDirectCollection(String.class)
-    private List<String> extraSpaces;
-
-    /** */
-    @GridToStringInclude
-    private int[] parts;
-
-    /** */
-    private int timeout;
-
-    /**
-     * Default constructor.
-     */
-    public GridQueryRequest() {
-        // No-op.
-    }
-
-    /**
-     * @param reqId Request ID.
-     * @param pageSize Page size.
-     * @param space Space.
-     * @param qrys Queries.
-     * @param topVer Topology version.
-     * @param extraSpaces All space names participating in query other than {@code space}.
-     * @param parts Optional partitions for unstable topology.
-     * @param timeout Timeout in millis.
-     */
-    public GridQueryRequest(
-        long reqId,
-        int pageSize,
-        String space,
-        Collection<GridCacheSqlQuery> qrys,
-        AffinityTopologyVersion topVer,
-        List<String> extraSpaces,
-        int[] parts,
-        int timeout) {
-        this.reqId = reqId;
-        this.pageSize = pageSize;
-        this.space = space;
-
-        this.qrys = qrys;
-        this.topVer = topVer;
-        this.extraSpaces = extraSpaces;
-        this.parts = parts;
-        this.timeout = timeout;
-    }
-
-    /**
-     * @param cp Copy from.
-     */
-    public GridQueryRequest(GridQueryRequest cp) {
-        this.reqId = cp.reqId;
-        this.pageSize = cp.pageSize;
-        this.space = cp.space;
-        this.qrys = cp.qrys;
-        this.topVer = cp.topVer;
-        this.extraSpaces = cp.extraSpaces;
-        this.parts = cp.parts;
-    }
-
-    /**
-     * @return All the needed partitions for {@link #space()} and {@link #extraSpaces()}.
-     */
-    public int[] partitions() {
-        return parts;
-    }
-
-    /**
-     * @param parts All the needed partitions for {@link #space()} and {@link #extraSpaces()}.
-     */
-    public void partitions(int[] parts) {
-        this.parts = parts;
-    }
-
-    /**
-     * @return All extra space names participating in query other than {@link #space()}.
-     */
-    public List<String> extraSpaces() {
-        return extraSpaces;
-    }
-
-    /**
-     * @return Topology version.
-     */
-    public AffinityTopologyVersion topologyVersion() {
-        return topVer;
-    }
-
-    /**
-     * @return Request ID.
-     */
-    public long requestId() {
-        return reqId;
-    }
-
-    /**
-     * @return Page size.
-     */
-    public int pageSize() {
-        return pageSize;
-    }
-
-    /**
-     * @return Space.
-     */
-    public String space() {
-        return space;
-    }
-
-    /**
-     * @return Timeout.
-     */
-    public int timeout() {
-        return this.timeout;
-    }
-
-    /**
-     * @return Queries.
-     */
-    public Collection<GridCacheSqlQuery> queries() {
-        return qrys;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void marshall(Marshaller m) {
-        if (F.isEmpty(qrys))
-            return;
-
-        for (GridCacheSqlQuery qry : qrys)
-            qry.marshall(m);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void unmarshall(Marshaller m, GridKernalContext ctx) {
-        if (F.isEmpty(qrys))
-            return;
-
-        for (GridCacheSqlQuery qry : qrys)
-            qry.unmarshall(m, ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridQueryRequest.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onAckReceived() {
-        // No-op.
-    }
-
-    /** {@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.writeInt("pageSize", pageSize))
-                    return false;
-
-                writer.incrementState();
-
-            case 1:
-                if (!writer.writeCollection("qrys", qrys, MessageCollectionItemType.MSG))
-                    return false;
-
-                writer.incrementState();
-
-            case 2:
-                if (!writer.writeLong("reqId", reqId))
-                    return false;
-
-                writer.incrementState();
-
-            case 3:
-                if (!writer.writeString("space", space))
-                    return false;
-
-                writer.incrementState();
-
-            case 4:
-                if (!writer.writeMessage("topVer", topVer))
-                    return false;
-
-                writer.incrementState();
-
-            case 5:
-                if (!writer.writeCollection("extraSpaces", extraSpaces, MessageCollectionItemType.STRING))
-                    return false;
-
-                writer.incrementState();
-
-            case 6:
-                if (!writer.writeIntArray("parts", parts))
-                    return false;
-
-                writer.incrementState();
-
-            case 7:
-                if (!writer.writeInt("timeout", timeout))
-                    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:
-                pageSize = reader.readInt("pageSize");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 1:
-                qrys = reader.readCollection("qrys", MessageCollectionItemType.MSG);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 2:
-                reqId = reader.readLong("reqId");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 3:
-                space = reader.readString("space");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 4:
-                topVer = reader.readMessage("topVer");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 5:
-                extraSpaces = reader.readCollection("extraSpaces", MessageCollectionItemType.STRING);
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 6:
-                parts = reader.readIntArray("parts");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 7:
-                timeout = reader.readInt("timeout");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return reader.afterMessageRead(GridQueryRequest.class);
-    }
-
-    /** {@inheritDoc} */
-    @Override public short directType() {
-        return 110;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        return 8;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/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 e0a5c7c..bf44723 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
@@ -20,14 +20,11 @@ package org.apache.ignite.internal.processors.service;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
@@ -92,7 +89,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.resources.IgniteInstanceResource;
@@ -101,7 +97,6 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.services.ServiceDescriptor;
-import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.thread.IgniteThreadFactory;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
@@ -121,12 +116,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
 @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter", "ConstantConditions"})
 public class GridServiceProcessor extends GridProcessorAdapter {
     /** */
-    public static final IgniteProductVersion LAZY_SERVICES_CFG_SINCE = IgniteProductVersion.fromString("1.5.22");
-
-    /** Versions that only compatible with each other, and from 1.5.33. */
-    private static final Set<IgniteProductVersion> SERVICE_TOP_CALLABLE_VER1;
-
-    /** */
     private final Boolean srvcCompatibilitySysProp;
 
     /** Time to wait before reassignment retries. */
@@ -173,31 +162,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Topology listener. */
     private DiscoveryEventListener topLsnr = new TopologyListener();
 
-    static {
-        Set<IgniteProductVersion> versions = new TreeSet<>(new Comparator<IgniteProductVersion>() {
-            @Override public int compare(final IgniteProductVersion o1, final IgniteProductVersion o2) {
-                return o1.compareToIgnoreTimestamp(o2);
-            }
-        });
-
-        versions.add(IgniteProductVersion.fromString("1.5.30"));
-        versions.add(IgniteProductVersion.fromString("1.5.31"));
-        versions.add(IgniteProductVersion.fromString("1.5.32"));
-        versions.add(IgniteProductVersion.fromString("1.6.3"));
-        versions.add(IgniteProductVersion.fromString("1.6.4"));
-        versions.add(IgniteProductVersion.fromString("1.6.5"));
-        versions.add(IgniteProductVersion.fromString("1.6.6"));
-        versions.add(IgniteProductVersion.fromString("1.6.7"));
-        versions.add(IgniteProductVersion.fromString("1.6.8"));
-        versions.add(IgniteProductVersion.fromString("1.6.9"));
-        versions.add(IgniteProductVersion.fromString("1.6.10"));
-        versions.add(IgniteProductVersion.fromString("1.7.0"));
-        versions.add(IgniteProductVersion.fromString("1.7.1"));
-        versions.add(IgniteProductVersion.fromString("1.7.2"));
-
-        SERVICE_TOP_CALLABLE_VER1 = Collections.unmodifiableSet(versions);
-    }
-
     /**
      * @param ctx Kernal context.
      */
@@ -700,25 +664,20 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param name Service name.
      * @param timeout If greater than 0 limits task execution time. Cannot be negative.
      * @return Service topology.
+     * @throws IgniteCheckedException On error.
      */
     public Map<UUID, Integer> serviceTopology(String name, long timeout) throws IgniteCheckedException {
         ClusterNode node = cache.affinity().mapKeyToNode(name);
 
-        if (node.version().compareTo(ServiceTopologyCallable.SINCE_VER) >= 0) {
-            final ServiceTopologyCallable call = new ServiceTopologyCallable(name);
+        final ServiceTopologyCallable call = new ServiceTopologyCallable(name);
 
-            call.serialize = SERVICE_TOP_CALLABLE_VER1.contains(node.version());
-
-            return ctx.closure().callAsyncNoFailover(
-                GridClosureCallMode.BROADCAST,
-                call,
-                Collections.singletonList(node),
-                false,
-                timeout
-            ).get();
-        }
-        else
-            return serviceTopology(cache, name);
+        return ctx.closure().callAsyncNoFailover(
+            GridClosureCallMode.BROADCAST,
+            call,
+            Collections.singletonList(node),
+            false,
+            timeout
+        ).get();
     }
 
     /**
@@ -1306,63 +1265,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         }
     }
 
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode node) {
-        IgniteNodeValidationResult res = super.validateNode(node);
-
-        if (res != null)
-            return res;
-
-        boolean rmtNodeIsOld = node.version().compareToIgnoreTimestamp(LAZY_SERVICES_CFG_SINCE) < 0;
-
-        if (!rmtNodeIsOld)
-            return null;
-
-        while (true) {
-            ServicesCompatibilityState state = compatibilityState.get();
-
-            if (state.srvcCompatibility)
-                return null;
-
-            // Remote node is old and services are in not compatible mode.
-            if (!state.used) {
-                if (!compatibilityState.compareAndSet(state, new ServicesCompatibilityState(true, false)))
-                    continue;
-
-                return null;
-            }
-
-            ClusterNode locNode = ctx.discovery().localNode();
-
-            return new IgniteNodeValidationResult(node.id(), "Local node uses IgniteServices and works in not " +
-                "compatible mode with old nodes (" + IGNITE_SERVICES_COMPATIBILITY_MODE + " system property can be " +
-                "set explicitly) [locNodeId=" + locNode.id() + ", rmtNodeId=" + node.id() + "]",
-                "Remote node uses IgniteServices and works in not compatible mode with old nodes " +
-                    IGNITE_SERVICES_COMPATIBILITY_MODE + " system property can be set explicitly" +
-                    "[locNodeId=" + node.id() + ", rmtNodeId=" + locNode.id() + "]");
-        }
-    }
-
     /**
      * @param nodes Remote nodes.
      */
     public void initCompatibilityMode(Collection<ClusterNode> nodes) {
-        boolean mode;
+        boolean mode = false;
 
-        if (srvcCompatibilitySysProp == null) {
-            boolean clusterHasOldNode = false;
-
-            for (ClusterNode n : nodes) {
-                if (n.version().compareToIgnoreTimestamp(LAZY_SERVICES_CFG_SINCE) < 0) {
-                    clusterHasOldNode = true;
-
-                    break;
-                }
-            }
-
-            mode = clusterHasOldNode;
-        }
-        else
+        if (srvcCompatibilitySysProp != null)
             mode = srvcCompatibilitySysProp;
 
         while (true) {
@@ -1881,12 +1790,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         private static final long serialVersionUID = 0L;
 
         /** */
-        private static final IgniteProductVersion SINCE_VER = IgniteProductVersion.fromString("1.5.7");
-
-        /** */
-        private static final String[] SER_FIELDS = {"waitedCacheInit", "jCtx", "log"};
-
-        /** */
         private final String svcName;
 
         /** */
@@ -1904,9 +1807,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         @LoggerResource
         private transient IgniteLogger log;
 
-        /** */
-        transient boolean serialize;
-
         /**
          * @param svcName Service name.
          */
@@ -1951,16 +1851,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
             return serviceTopology(cache, svcName);
         }
-
-        /**
-         * @param self Instance of current class before serialization.
-         * @param ver Sender job version.
-         * @return List of serializable transient fields.
-         */
-        @SuppressWarnings("unused")
-        private static String[] serializableTransient(ServiceTopologyCallable self, IgniteProductVersion ver) {
-            return (self != null && self.serialize) || (ver != null && SERVICE_TOP_CALLABLE_VER1.contains(ver)) ? SER_FIELDS : null;
-        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index a440ac3..ec6a350 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.internal.visor.cache;
 
 import java.io.Serializable;
-import java.util.Collection;
-import java.util.Collections;
 import java.util.Iterator;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.IgniteCheckedException;
@@ -27,14 +25,14 @@ import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
-import org.apache.ignite.internal.util.lang.IgnitePair;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
@@ -91,11 +89,22 @@ public class VisorCache implements Serializable, LessNamingBean {
     /** Number of partitions. */
     private int partitions;
 
-    /** @deprecated Needed only for backward compatibility. */
-    private Collection<IgnitePair<Integer>> primaryPartitions;
+    /**
+     * Flag indicating that cache has near cache.
+     */
+    private boolean near;
+
+    /** Number of primary entries in offheap. */
+    private int offHeapPrimaryEntriesCnt;
 
-    /** @deprecated Needed only for backward compatibility. */
-    private Collection<IgnitePair<Integer>> backupPartitions;
+    /** Number of backup entries in offheap. */
+    private int offHeapBackupEntriesCnt;
+
+    /** Number of primary entries in swap. */
+    private int swapPrimaryEntriesCnt;
+
+    /** Number of backup entries in swap. */
+    private int swapBackupEntriesCnt;
 
     /** Cache metrics. */
     private VisorCacheMetrics metrics;
@@ -130,9 +139,6 @@ public class VisorCache implements Serializable, LessNamingBean {
             swapKeys = -1;
         }
 
-        primaryPartitions = Collections.emptyList();
-        backupPartitions = Collections.emptyList();
-
         CacheConfiguration cfg = ca.configuration();
 
         mode = cfg.getCacheMode();
@@ -151,11 +157,8 @@ public class VisorCache implements Serializable, LessNamingBean {
             if (dca != null) {
                 GridDhtPartitionTopology top = dca.topology();
 
-                if (cfg.getCacheMode() != CacheMode.LOCAL && cfg.getBackups() > 0) {
-                    GridDhtPartitionMap2 map2 = top.localPartitionMap();
-
-                    partitionsMap = new GridDhtPartitionMap(map2.nodeId(), map2.updateSequence(), map2.map());
-                }
+                if (cfg.getCacheMode() != CacheMode.LOCAL && cfg.getBackups() > 0)
+                    partitionsMap = top.localPartitionMap();
             }
         }
 
@@ -168,6 +171,15 @@ public class VisorCache implements Serializable, LessNamingBean {
         offHeapEntriesCnt = ca.offHeapEntriesCount();
         partitions = ca.affinity().partitions();
         metrics = new VisorCacheMetrics().from(ignite, cacheName);
+        near = ca.context().isNear();
+
+        GridCacheSwapManager swap = ca.context().swap();
+
+        offHeapPrimaryEntriesCnt = swap.offheapEntriesCount(true, false, AffinityTopologyVersion.NONE);
+        offHeapBackupEntriesCnt = swap.offheapEntriesCount(false, true, AffinityTopologyVersion.NONE);
+
+        swapPrimaryEntriesCnt = swap.swapEntriesCount(true, false, AffinityTopologyVersion.NONE);
+        swapBackupEntriesCnt = swap.swapEntriesCount(false, true, AffinityTopologyVersion.NONE);
 
         estimateMemorySize(ignite, ca, sample);
 
@@ -230,9 +242,12 @@ public class VisorCache implements Serializable, LessNamingBean {
             c.swapSize = swapSize;
             c.swapKeys = swapKeys;
             c.partitions = partitions;
-            c.primaryPartitions = Collections.emptyList();
-            c.backupPartitions = Collections.emptyList();
             c.metrics = metrics;
+            c.near = near;
+            c.offHeapPrimaryEntriesCnt = offHeapPrimaryEntriesCnt;
+            c.offHeapBackupEntriesCnt = offHeapBackupEntriesCnt;
+            c.swapPrimaryEntriesCnt = swapPrimaryEntriesCnt;
+            c.swapBackupEntriesCnt = swapBackupEntriesCnt;
         }
 
         return c;
@@ -353,17 +368,38 @@ public class VisorCache implements Serializable, LessNamingBean {
     }
 
     /**
-     * @deprecated Needed only for backward compatibility.
+     * @return {@code true} if cache has near cache.
+     */
+    public boolean near() {
+        return near;
+    }
+
+    /**
+     * @return Off-heap heap primary entries count.
+     */
+    public int offHeapPrimaryEntriesCount() {
+        return offHeapPrimaryEntriesCnt;
+    }
+
+    /**
+     * @return Off-heap heap backup entries count.
+     */
+    public int offHeapBackupEntriesCount() {
+        return offHeapBackupEntriesCnt;
+    }
+
+    /**
+     * @return Swap primary entries count.
      */
-    public Collection<IgnitePair<Integer>> primaryPartitions() {
-        return primaryPartitions;
+    public int swapPrimaryEntriesCount() {
+        return swapPrimaryEntriesCnt;
     }
 
     /**
-     * @deprecated Needed only for backward compatibility.
+     * @return Swap backup entries count.
      */
-    public Collection<IgnitePair<Integer>> backupPartitions() {
-        return backupPartitions;
+    public int swapBackupEntriesCount() {
+        return swapBackupEntriesCnt;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
index c779051..586fa87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAggregatedMetrics.java
@@ -206,7 +206,7 @@ public class VisorCacheAggregatedMetrics implements Serializable, LessNamingBean
      * @return Off heap entries count.
      */
     private long offHeapEntriesCount(VisorCacheMetrics metric) {
-        return metric instanceof VisorCacheMetricsV2 ? ((VisorCacheMetricsV2) metric).offHeapEntriesCount() : 0;
+        return metric.offHeapEntriesCount();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
index 869a12c..cf2ba7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfiguration.java
@@ -24,13 +24,11 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
-import org.apache.ignite.lang.IgniteProductVersion;
 
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
 
@@ -41,9 +39,6 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** */
-    private static final IgniteProductVersion VER_1_4_1 = IgniteProductVersion.fromString("1.4.1");
-
     /** Cache name. */
     private String name;
 
@@ -158,20 +153,9 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
         nearCfg = VisorCacheNearConfiguration.from(ccfg);
         dfltCfg = VisorCacheDefaultConfiguration.from(ccfg);
 
-        boolean compatibility = false;
-
-        for (ClusterNode node : ignite.cluster().nodes()) {
-            if (node.version().compareToIgnoreTimestamp(VER_1_4_1) <= 0) {
-                compatibility = true;
-
-                break;
-            }
-        }
-
-        storeCfg = (compatibility ? new VisorCacheStoreConfiguration() : new VisorCacheStoreConfigurationV2())
-            .from(ignite, ccfg);
+        storeCfg = new VisorCacheStoreConfiguration().from(ignite, ccfg);
 
-        qryCfg = (compatibility ? new VisorCacheQueryConfiguration() : new VisorCacheQueryConfigurationV2()).from(ccfg);
+        qryCfg = new VisorCacheQueryConfiguration().from(ccfg);
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
index 1204cbc..507aacd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.CacheMetrics;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -156,6 +157,12 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
     /** Number of cached rolled back DHT transaction IDs. */
     private int txDhtRolledbackVersionsSize;
 
+    /** Memory size allocated in off-heap. */
+    private long offHeapAllocatedSize;
+
+    /** Number of cache entries stored in off-heap memory. */
+    private long offHeapEntriesCount;
+
     /**
      * Calculate rate of metric per second.
      *
@@ -228,6 +235,11 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
         txDhtCommittedVersionsSize = m.getTxDhtCommittedVersionsSize();
         txDhtRolledbackVersionsSize = m.getTxDhtRolledbackVersionsSize();
 
+        GridCacheAdapter<Object, Object> ca = cacheProcessor.internalCache(cacheName);
+
+        offHeapAllocatedSize = ca.offHeapAllocatedSize();
+        offHeapEntriesCount = ca.offHeapEntriesCount();
+
         return this;
     }
 
@@ -520,6 +532,20 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
         return txDhtRolledbackVersionsSize;
     }
 
+    /**
+     * @return Memory size allocated in off-heap.
+     */
+    public long offHeapAllocatedSize() {
+        return offHeapAllocatedSize;
+    }
+
+    /**
+     * @return Number of cache entries stored in off-heap memory.
+     */
+    public long offHeapEntriesCount() {
+        return offHeapEntriesCount;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheMetrics.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
index 4dd1e28..4fd85de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
@@ -21,7 +21,6 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
@@ -31,7 +30,6 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorMultiNodeTask;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteProductVersion;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -83,9 +81,6 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** */
-        private static final IgniteProductVersion V2_SINCE = IgniteProductVersion.fromString("1.5.8");
-
         /**
          * Create job with given argument.
          *
@@ -120,18 +115,7 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
                 if (ca.context().started()) {
                     String cacheName = ca.getName();
 
-                    boolean compatibilityMode = false;
-
-                    for (ClusterNode node : ignite.cluster().nodes()) {
-                        if (node.version().compareToIgnoreTimestamp(V2_SINCE) < 0) {
-                            compatibilityMode = true;
-
-                            break;
-                        }
-                    }
-
-                    VisorCacheMetrics cm = (compatibilityMode ? new VisorCacheMetrics() : new VisorCacheMetricsV2())
-                            .from(ignite, cacheName);
+                    VisorCacheMetrics cm = new VisorCacheMetrics().from(ignite, cacheName);
 
                     if ((allCaches || cacheNames.contains(cacheName)) && (showSysCaches || !cm.system()))
                         res.add(cm);

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsV2.java
deleted file mode 100644
index 2376db0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsV2.java
+++ /dev/null
@@ -1,66 +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.visor.cache;
-
-import org.apache.ignite.cache.CacheMetrics;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
-
-/**
- * Data transfer object for {@link CacheMetrics}.
- */
-public class VisorCacheMetricsV2 extends VisorCacheMetrics {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Memory size allocated in off-heap. */
-    private long offHeapAllocatedSize;
-
-    /** Number of cache entries stored in off-heap memory. */
-    private long offHeapEntriesCount;
-
-    /** {@inheritDoc} */
-    @Override
-    public VisorCacheMetrics from(IgniteEx ignite, String cacheName) {
-        super.from(ignite, cacheName);
-
-        GridCacheProcessor cacheProcessor = ignite.context().cache();
-
-        GridCacheAdapter<Object, Object> c = cacheProcessor.internalCache(cacheName);
-
-        offHeapAllocatedSize = c.offHeapAllocatedSize();
-        offHeapEntriesCount = c.offHeapEntriesCount();
-
-        return this;
-    }
-
-    /**
-     * @return Memory size allocated in off-heap.
-     */
-    public long offHeapAllocatedSize() {
-        return offHeapAllocatedSize;
-    }
-
-    /**
-     * @return Number of cache entries stored in off-heap memory.
-     */
-    public long offHeapEntriesCount() {
-        return offHeapEntriesCount;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
index e0d1e72..d1a0cdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
@@ -44,6 +44,16 @@ public class VisorCacheQueryConfiguration implements Serializable, LessNamingBea
     /** */
     private int sqlOnheapRowCacheSize;
 
+    /** */
+    private String sqlSchema;
+
+    /**
+     * @return Schema name, which is used by SQL engine for SQL statements generation.
+     */
+    public String sqlSchema() {
+        return sqlSchema;
+    }
+
     /**
      * @param clss Classes to compact.
      */
@@ -71,6 +81,7 @@ public class VisorCacheQueryConfiguration implements Serializable, LessNamingBea
         sqlEscapeAll = ccfg.isSqlEscapeAll();
         indexedTypes = compactClasses(ccfg.getIndexedTypes());
         sqlOnheapRowCacheSize = ccfg.getSqlOnheapRowCacheSize();
+        sqlSchema = ccfg.getSqlSchema();
 
         return this;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfigurationV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfigurationV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfigurationV2.java
deleted file mode 100644
index e914b73..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfigurationV2.java
+++ /dev/null
@@ -1,47 +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.visor.cache;
-
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/**
- * Data transfer object for cache query configuration data.
- */
-public class VisorCacheQueryConfigurationV2 extends VisorCacheQueryConfiguration {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private String sqlSchema;
-
-    /**
-     * @return Schema name, which is used by SQL engine for SQL statements generation.
-     */
-    public String sqlSchema() {
-        return sqlSchema;
-    }
-
-    /** {@inheritDoc} */
-    @Override public VisorCacheQueryConfiguration from(CacheConfiguration ccfg) {
-        super.from(ccfg);
-
-        sqlSchema = ccfg.getSqlSchema();
-
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
index 5d3e1e1..fd3d980 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
@@ -66,6 +66,9 @@ public class VisorCacheStoreConfiguration implements Serializable, LessNamingBea
     /** Number of threads that will perform cache flushing. */
     private int flushThreadCnt;
 
+    /** Keep binary in store flag. */
+    private boolean storeKeepBinary;
+
     /**
      * @param ignite Ignite instance.
      * @param ccfg Cache configuration.
@@ -90,6 +93,8 @@ public class VisorCacheStoreConfiguration implements Serializable, LessNamingBea
         flushSz = ccfg.getWriteBehindFlushSize();
         flushThreadCnt = ccfg.getWriteBehindFlushThreadCount();
 
+        storeKeepBinary = ccfg.isStoreKeepBinary();
+
         return this;
     }
 
@@ -170,6 +175,13 @@ public class VisorCacheStoreConfiguration implements Serializable, LessNamingBea
         return flushThreadCnt;
     }
 
+    /**
+     * @return Keep binary in store flag.
+     */
+    public boolean storeKeepBinary() {
+        return storeKeepBinary;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheStoreConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfigurationV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfigurationV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfigurationV2.java
deleted file mode 100644
index 8595177..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfigurationV2.java
+++ /dev/null
@@ -1,48 +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.visor.cache;
-
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.IgniteEx;
-
-/**
- * Data transfer object for cache store configuration properties.
- */
-public class VisorCacheStoreConfigurationV2 extends VisorCacheStoreConfiguration {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Keep binary in store flag. */
-    private boolean storeKeepBinary;
-
-    /** {@inheritDoc} */
-    @Override public VisorCacheStoreConfiguration from(IgniteEx ignite, CacheConfiguration ccfg) {
-        super.from(ignite, ccfg);
-
-        storeKeepBinary = ccfg.isStoreKeepBinary();
-
-        return this;
-    }
-
-    /**
-     * @return Keep binary in store flag.
-     */
-    public boolean storeKeepBinary() {
-        return storeKeepBinary;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
deleted file mode 100644
index 61551cc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
+++ /dev/null
@@ -1,73 +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.visor.cache;
-
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-
-/**
- * Data transfer object for {@link IgniteCache}.
- */
-public class VisorCacheV2 extends VisorCache {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Flag indicating that cache has near cache.
-     */
-    private boolean near;
-
-    /** {@inheritDoc} */
-    @Override public VisorCache from(IgniteEx ignite, String cacheName, int sample) throws IgniteCheckedException {
-        VisorCache c = super.from(ignite, cacheName, sample);
-
-        if (c != null && c instanceof VisorCacheV2) {
-            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
-
-            // Process only started caches.
-            if (ca != null && ca.context().started())
-                ((VisorCacheV2)c).near = ca.context().isNear();
-        }
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected VisorCache initHistory(VisorCache c) {
-        super.initHistory(c);
-
-        if (c instanceof VisorCacheV2)
-            ((VisorCacheV2) c).near = near;
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override public VisorCache history() {
-        return initHistory(new VisorCacheV2());
-    }
-
-    /**
-     * @return {@code true} if cache has near cache.
-     */
-    public boolean near() {
-        return near;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
deleted file mode 100644
index fab37e3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV3.java
+++ /dev/null
@@ -1,52 +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.visor.cache;
-
-import java.util.Collection;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.internal.util.lang.GridTuple3;
-
-/**
- * Data transfer object for {@link IgniteCache}.
- *
- * @deprecated Needed only for backward compatibility.
- */
-public class VisorCacheV3 extends VisorCacheV2 {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** @deprecated Needed only for backward compatibility. */
-    private Collection<GridTuple3<Integer, Long, Long>> primaryPartsOffheapSwap;
-
-    /** @deprecated Needed only for backward compatibility. */
-    private Collection<GridTuple3<Integer, Long, Long>> backupPartsOffheapSwap;
-
-    /**
-     * @deprecated Needed only for backward compatibility.
-     */
-    public Collection<GridTuple3<Integer, Long, Long>> primaryPartitionsOffheapSwap() {
-        return primaryPartsOffheapSwap;
-    }
-
-    /**
-     * @deprecated Needed only for backward compatibility.
-     */
-    public Collection<GridTuple3<Integer, Long, Long>> backupPartitionsOffheapSwap() {
-        return backupPartsOffheapSwap;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV4.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV4.java
deleted file mode 100644
index a5f66be..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV4.java
+++ /dev/null
@@ -1,124 +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.visor.cache;
-
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapManager;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Data transfer object for {@link IgniteCache}.
- */
-public class VisorCacheV4 extends VisorCacheV2 {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Number of primary entries in offheap. */
-    private int offHeapPrimaryEntriesCnt;
-
-    /** Number of backup entries in offheap. */
-    private int offHeapBackupEntriesCnt;
-
-    /** Number of primary entries in swap. */
-    private int swapPrimaryEntriesCnt;
-
-    /** Number of backup entries in swap. */
-    private int swapBackupEntriesCnt;
-
-    /** {@inheritDoc} */
-    @Override public VisorCache from(IgniteEx ignite, String cacheName, int sample) throws IgniteCheckedException {
-        VisorCache c = super.from(ignite, cacheName, sample);
-
-        if (c != null && c instanceof VisorCacheV4) {
-            VisorCacheV4 cacheV4 = (VisorCacheV4)c;
-
-            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
-
-            // Process only started caches.
-            if (ca != null && ca.context().started()) {
-                GridCacheSwapManager swap = ca.context().swap();
-
-                cacheV4.offHeapPrimaryEntriesCnt = swap.offheapEntriesCount(true, false, AffinityTopologyVersion.NONE);
-                cacheV4.offHeapBackupEntriesCnt = swap.offheapEntriesCount(false, true, AffinityTopologyVersion.NONE);
-
-                cacheV4.swapPrimaryEntriesCnt = swap.swapEntriesCount(true, false, AffinityTopologyVersion.NONE);
-                cacheV4.swapBackupEntriesCnt = swap.swapEntriesCount(false, true, AffinityTopologyVersion.NONE);
-            }
-        }
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected VisorCache initHistory(VisorCache c) {
-        super.initHistory(c);
-
-        if (c instanceof VisorCacheV4) {
-            VisorCacheV4 cacheV4 = (VisorCacheV4)c;
-
-            cacheV4.offHeapPrimaryEntriesCnt = offHeapPrimaryEntriesCnt;
-            cacheV4.offHeapBackupEntriesCnt = offHeapBackupEntriesCnt;
-            cacheV4.swapPrimaryEntriesCnt = swapPrimaryEntriesCnt;
-            cacheV4.swapBackupEntriesCnt = swapBackupEntriesCnt;
-        }
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override public VisorCache history() {
-        return initHistory(new VisorCacheV4());
-    }
-
-    /**
-     * @return Off-heap heap primary entries count.
-     */
-    public int offHeapPrimaryEntriesCount() {
-        return offHeapPrimaryEntriesCnt;
-    }
-
-    /**
-     * @return Off-heap heap backup entries count.
-     */
-    public int offHeapBackupEntriesCount() {
-        return offHeapBackupEntriesCnt;
-    }
-
-    /**
-     * @return Swap primary entries count.
-     */
-    public int swapPrimaryEntriesCount() {
-        return swapPrimaryEntriesCnt;
-    }
-
-    /**
-     * @return Swap backup entries count.
-     */
-    public int swapBackupEntriesCount() {
-        return swapBackupEntriesCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheV4.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEvent.java
index 7d970de..bad6966 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEvent.java
@@ -38,6 +38,9 @@ public class VisorGridDiscoveryEvent extends VisorGridEvent {
     /** If node that caused this event is daemon. */
     private final boolean isDaemon;
 
+    /** Topology version. */
+    private final long topVer;
+
     /**
      * Create event with given parameters.
      *
@@ -51,6 +54,7 @@ public class VisorGridDiscoveryEvent extends VisorGridEvent {
      * @param evtNodeId Event node id.
      * @param addr Event node address.
      * @param isDaemon If event node is daemon on not.
+     * @param topVer Topology version.
      */
     public VisorGridDiscoveryEvent(
         int typeId,
@@ -62,13 +66,15 @@ public class VisorGridDiscoveryEvent extends VisorGridEvent {
         String shortDisplay,
         UUID evtNodeId,
         String addr,
-        boolean isDaemon
+        boolean isDaemon,
+        long topVer
     ) {
         super(typeId, id, name, nid, ts, msg, shortDisplay);
 
         this.evtNodeId = evtNodeId;
         this.addr = addr;
         this.isDaemon = isDaemon;
+        this.topVer = topVer;
     }
 
     /**
@@ -92,8 +98,16 @@ public class VisorGridDiscoveryEvent extends VisorGridEvent {
         return isDaemon;
     }
 
+    /**
+     * @return Topology version or {@code 0} if configured discovery SPI implementation
+     *      does not support versioning.
+     **/
+    public long topologyVersion() {
+        return topVer;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorGridDiscoveryEvent.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java
deleted file mode 100644
index b66aacf..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/event/VisorGridDiscoveryEventV2.java
+++ /dev/null
@@ -1,80 +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.visor.event;
-
-import java.util.UUID;
-import org.apache.ignite.internal.util.typedef.internal.S;
-import org.apache.ignite.lang.IgniteUuid;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Lightweight counterpart for {@link org.apache.ignite.events.DiscoveryEvent}.
- */
-public class VisorGridDiscoveryEventV2 extends VisorGridDiscoveryEvent {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Topology version. */
-    private final long topVer;
-
-    /**
-     * Create event with given parameters.
-     *
-     * @param typeId Event type.
-     * @param id Event id.
-     * @param name Event name.
-     * @param nid Event node ID.
-     * @param ts Event timestamp.
-     * @param msg Event message.
-     * @param shortDisplay Shortened version of {@code toString()} result.
-     * @param evtNodeId Event node id.
-     * @param addr Event node address.
-     * @param isDaemon If event node is daemon on not.
-     * @param topVer Topology version.
-     */
-    public VisorGridDiscoveryEventV2(
-        int typeId,
-        IgniteUuid id,
-        String name,
-        UUID nid,
-        long ts,
-        @Nullable String msg,
-        String shortDisplay,
-        UUID evtNodeId,
-        String addr,
-        boolean isDaemon,
-        long topVer
-    ) {
-        super(typeId, id, name, nid, ts, msg, shortDisplay, evtNodeId, addr, isDaemon);
-
-        this.topVer = topVer;
-    }
-
-    /**
-     * @return Topology version or {@code 0} if configured discovery SPI implementation
-     *      does not support versioning.
-     **/
-    public long topologyVersion() {
-        return topVer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorGridDiscoveryEventV2.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/488b25e1/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index f45725f..0617e70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -30,19 +30,14 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.cache.VisorCache;
-import org.apache.ignite.internal.visor.cache.VisorCacheV2;
-import org.apache.ignite.internal.visor.cache.VisorCacheV3;
-import org.apache.ignite.internal.visor.cache.VisorCacheV4;
 import org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
 import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
-import org.apache.ignite.lang.IgniteProductVersion;
 
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isIgfsCache;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isSystemCache;
 import static org.apache.ignite.internal.visor.compute.VisorComputeMonitoringHolder.COMPUTE_MONITORING_HOLDER_KEY;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.EVT_MAPPER;
-import static org.apache.ignite.internal.visor.util.VisorTaskUtils.EVT_MAPPER_V2;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.VISOR_TASK_EVTS;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.checkExplicitTaskMonitoring;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.collectEvents;
@@ -55,18 +50,6 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** */
-    private static final IgniteProductVersion VER_1_4_1 = IgniteProductVersion.fromString("1.4.1");
-
-    /** */
-    private static final IgniteProductVersion VER_1_5_10 = IgniteProductVersion.fromString("1.5.10");
-
-    /** */
-    private static final IgniteProductVersion VER_1_5_26 = IgniteProductVersion.fromString("1.5.26");
-
-    /** */
-    protected static final IgniteProductVersion VER_1_7_2 = IgniteProductVersion.fromString("1.7.2");
-
     /**
      * Create job with given argument.
      *
@@ -87,8 +70,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
      */
     protected void events0(VisorNodeDataCollectorJobResult res, String evtOrderKey, String evtThrottleCntrKey,
         final boolean all) {
-        res.events().addAll(collectEvents(ignite, evtOrderKey, evtThrottleCntrKey, all,
-            compatibleWith(VER_1_7_2) ? EVT_MAPPER_V2 : EVT_MAPPER));
+        res.events().addAll(collectEvents(ignite, evtOrderKey, evtThrottleCntrKey, all, EVT_MAPPER));
     }
 
     /**
@@ -135,34 +117,6 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
     }
 
     /**
-     * @param ver Version to check.
-     * @return {@code true} if found at least one compatible node with specified version.
-     */
-    protected boolean compatibleWith(IgniteProductVersion ver) {
-        for (ClusterNode node : ignite.cluster().nodes())
-            if (node.version().compareToIgnoreTimestamp(ver) <= 0)
-                return true;
-
-        return false;
-    }
-
-    /**
-     * @return Compatible {@link VisorCache} instance.
-     */
-    private VisorCache createVisorCache() {
-        if (compatibleWith(VER_1_4_1))
-            return new VisorCache();
-
-        if (compatibleWith(VER_1_5_10))
-            return new VisorCacheV2();
-
-        if (compatibleWith(VER_1_5_26))
-            return new VisorCacheV3();
-
-        return new VisorCacheV4();
-    }
-
-    /**
      * @param cacheName Cache name to check.
      * @return {@code true} if cache on local node is not a data cache or near cache disabled.
      */
@@ -189,11 +143,11 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                     continue;
 
                 if (arg.systemCaches() ||
-                    !(isSystemCache(cacheName) || isIgfsCache(ignite.configuration(), cacheName))) {
+                    !(isSystemCache(cacheName) || isIgfsCache(cfg, cacheName))) {
                     long start0 = U.currentTimeMillis();
 
                     try {
-                        VisorCache cache = createVisorCache().from(ignite, cacheName, arg.sample());
+                        VisorCache cache = new VisorCache().from(ignite, cacheName, arg.sample());
 
                         if (cache != null)
                             res.caches().add(cache);