You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2017/04/06 08:01:27 UTC

[7/8] ignite git commit: GG-11732 Merged with master.

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
index 7556e7c..389f483 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
@@ -18,13 +18,13 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorOneNodeTask;
-import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -34,7 +34,7 @@ import org.apache.ignite.resources.JobContextResource;
  * Task that clears specified caches on specified node.
  */
 @GridInternal
-public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<Integer, Integer>> {
+public class VisorCacheClearTask extends VisorOneNodeTask<String, VisorCacheClearTaskResult> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -46,7 +46,7 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
     /**
      * Job that clear specified caches.
      */
-    private static class VisorCacheClearJob extends VisorJob<String, IgniteBiTuple<Integer, Integer>> {
+    private static class VisorCacheClearJob extends VisorJob<String, VisorCacheClearTaskResult> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -54,10 +54,10 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         private final String cacheName;
 
         /** */
-        private final IgniteInClosure<IgniteFuture<Integer>> lsnr;
+        private final IgniteInClosure<IgniteFuture<Long>> lsnr;
 
         /** */
-        private IgniteFuture<Integer>[] futs;
+        private IgniteFuture<Long>[] futs;
 
         /** */
         @JobContextResource
@@ -74,11 +74,11 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
 
             this.cacheName = cacheName;
 
-            lsnr = new IgniteInClosure<IgniteFuture<Integer>>() {
+            lsnr = new IgniteInClosure<IgniteFuture<Long>>() {
                 /** */
                 private static final long serialVersionUID = 0L;
 
-                @Override public void apply(IgniteFuture<Integer> f) {
+                @Override public void apply(IgniteFuture<Long> f) {
                     assert futs[0].isDone();
                     assert futs[1] == null || futs[1].isDone();
                     assert futs[2] == null || futs[2].isDone();
@@ -89,11 +89,17 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /**
-         * @param fut Future for asynchronous cache operation.
+         * @param subJob Sub job to execute asynchronously.
          * @param idx Index.
          * @return {@code true} If subJob was not completed and this job should be suspended.
          */
-        private boolean callAsync(IgniteFuture<Integer> fut, int idx) {
+        private boolean callAsync(IgniteCallable<Long> subJob, int idx) {
+            IgniteCompute compute = ignite.compute(ignite.cluster().forCacheNodes(cacheName)).withAsync();
+
+            compute.call(subJob);
+
+            IgniteFuture<Long> fut = compute.future();
+
             futs[idx] = fut;
 
             if (fut.isDone())
@@ -107,32 +113,26 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /** {@inheritDoc} */
-        @Override protected IgniteBiTuple<Integer, Integer> run(final String cacheName) {
+        @Override protected VisorCacheClearTaskResult run(final String cacheName) {
             if (futs == null)
                 futs = new IgniteFuture[3];
 
             if (futs[0] == null || futs[1] == null || futs[2] == null) {
                 IgniteCache cache = ignite.cache(cacheName);
 
-                if (futs[0] == null) {
-                    if (callAsync(cache.sizeAsync(CachePeekMode.PRIMARY), 0))
-                        return null;
-                }
+                if (futs[0] == null && callAsync(new VisorCacheSizeCallable(cache), 0))
+                    return null;
 
-                if (futs[1] == null) {
-                    if (callAsync(cache.clearAsync(), 1))
-                        return null;
-                }
-                
-                if (futs[2] == null) {
-                    if (callAsync(cache.sizeAsync(CachePeekMode.PRIMARY), 2))
-                        return null;
-                }
+                if (futs[1] == null && callAsync(new VisorCacheClearCallable(cache), 1))
+                    return null;
+
+                if (futs[2] == null && callAsync(new VisorCacheSizeCallable(cache), 2))
+                    return null;
             }
 
             assert futs[0].isDone() && futs[1].isDone() && futs[2].isDone();
 
-            return new IgniteBiTuple<>(futs[0].get(), futs[2].get());
+            return new VisorCacheClearTaskResult(futs[0].get(), futs[2].get());
         }
 
         /** {@inheritDoc} */
@@ -143,11 +143,9 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
 
     /**
      * Callable to get cache size.
-     *
-     * @deprecated This class needed only for compatibility.
      */
-    @GridInternal @Deprecated
-    private static class VisorCacheSizeCallable implements IgniteCallable<Integer> {
+    @GridInternal
+    private static class VisorCacheSizeCallable implements IgniteCallable<Long> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -162,18 +160,16 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
-            return cache.size(CachePeekMode.PRIMARY);
+        @Override public Long call() throws Exception {
+            return cache.sizeLong(CachePeekMode.PRIMARY);
         }
     }
 
     /**
      * Callable to clear cache.
-     *
-     * @deprecated This class needed only for compatibility.
      */
-    @GridInternal @Deprecated
-    private static class VisorCacheClearCallable implements IgniteCallable<Integer> {
+    @GridInternal
+    private static class VisorCacheClearCallable implements IgniteCallable<Long> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -188,10 +184,10 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
+        @Override public Long call() throws Exception {
             cache.clear();
 
-            return 0;
+            return 0L;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTaskResult.java
new file mode 100644
index 0000000..aba31f1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTaskResult.java
@@ -0,0 +1,85 @@
+/*
+ * 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.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * Result for {@link VisorCacheClearTask}.
+ */
+public class VisorCacheClearTaskResult extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Cache size before clearing. */
+    private long sizeBefore;
+
+    /** Cache size after clearing. */
+    private long sizeAfter;
+
+    /**
+     * Default constructor.
+     */
+    public VisorCacheClearTaskResult() {
+        // No-op.
+    }
+
+    /**
+     * @param sizeBefore Cache size before clearing.
+     * @param sizeAfter Cache size after clearing.
+     */
+    public VisorCacheClearTaskResult(long sizeBefore, long sizeAfter) {
+        this.sizeBefore = sizeBefore;
+        this.sizeAfter = sizeAfter;
+    }
+
+    /**
+     * @return Cache size before clearing.
+     */
+    public long getSizeBefore() {
+        return sizeBefore;
+    }
+
+    /**
+     * @return Cache size after clearing.
+     */
+    public long getSizeAfter() {
+        return sizeAfter;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeLong(sizeBefore);
+        out.writeLong(sizeAfter);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        sizeBefore = in.readLong();
+        sizeAfter = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheClearTaskResult.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/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 30973b0..28eda46 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
@@ -17,17 +17,19 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.Collection;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 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.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.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
@@ -35,7 +37,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
 /**
  * Data transfer object for cache configuration properties.
  */
-public class VisorCacheConfiguration implements Serializable, LessNamingBean {
+public class VisorCacheConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -57,9 +59,6 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
     /** Write synchronization mode. */
     private CacheWriteSynchronizationMode writeSynchronizationMode;
 
-    /** Swap enabled flag. */
-    private boolean swapEnabled;
-
     /** Invalidate. */
     private boolean invalidate;
 
@@ -72,12 +71,12 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
     /** Max concurrent async operations. */
     private int maxConcurrentAsyncOps;
 
-    /** Memory mode. */
-    private CacheMemoryMode memoryMode;
-
     /** Cache interceptor. */
     private String interceptor;
 
+    /** Gets default lock acquisition timeout. */
+    private long dfltLockTimeout;
+
     /** Cache affinityCfg config. */
     private VisorCacheAffinityConfiguration affinityCfg;
 
@@ -90,9 +89,6 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
     /** Near cache config. */
     private VisorCacheNearConfiguration nearCfg;
 
-    /** Default config. */
-    private VisorCacheDefaultConfiguration dfltCfg;
-
     /** Store config. */
     private VisorCacheStoreConfiguration storeCfg;
 
@@ -121,24 +117,31 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
     private boolean sys;
 
     /**
+     * Default constructor.
+     */
+    public VisorCacheConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object for cache configuration properties.
+     *
      * @param ignite Grid.
      * @param ccfg Cache configuration.
-     * @return Data transfer object for cache configuration properties.
      */
-    public VisorCacheConfiguration from(IgniteEx ignite, CacheConfiguration ccfg) {
+    public VisorCacheConfiguration(IgniteEx ignite, CacheConfiguration ccfg) {
         name = ccfg.getName();
         mode = ccfg.getCacheMode();
         atomicityMode = ccfg.getAtomicityMode();
         atomicWriteOrderMode = ccfg.getAtomicWriteOrderMode();
         eagerTtl = ccfg.isEagerTtl();
         writeSynchronizationMode = ccfg.getWriteSynchronizationMode();
-        swapEnabled = ccfg.isSwapEnabled();
         invalidate = ccfg.isInvalidate();
         startSize = ccfg.getStartSize();
         offHeapMaxMemory = ccfg.getOffHeapMaxMemory();
         maxConcurrentAsyncOps = ccfg.getMaxConcurrentAsyncOperations();
-        memoryMode = ccfg.getMemoryMode();
         interceptor = compactClass(ccfg.getInterceptor());
+        dfltLockTimeout = ccfg.getDefaultLockTimeout();
         typeMeta = VisorCacheTypeMetadata.list(ccfg.getQueryEntities(), ccfg.getCacheStoreFactory());
         statisticsEnabled = ccfg.isStatisticsEnabled();
         mgmtEnabled = ccfg.isManagementEnabled();
@@ -147,209 +150,250 @@ public class VisorCacheConfiguration implements Serializable, LessNamingBean {
         expiryPlcFactory = compactClass(ccfg.getExpiryPolicyFactory());
         sys = ignite.context().cache().systemCache(ccfg.getName());
 
-        affinityCfg = VisorCacheAffinityConfiguration.from(ccfg);
-        rebalanceCfg = VisorCacheRebalanceConfiguration.from(ccfg);
-        evictCfg = VisorCacheEvictionConfiguration.from(ccfg);
-        nearCfg = VisorCacheNearConfiguration.from(ccfg);
-        dfltCfg = VisorCacheDefaultConfiguration.from(ccfg);
-
-        storeCfg = new VisorCacheStoreConfiguration().from(ignite, ccfg);
+        affinityCfg = new VisorCacheAffinityConfiguration(ccfg);
+        rebalanceCfg = new VisorCacheRebalanceConfiguration(ccfg);
+        evictCfg = new VisorCacheEvictionConfiguration(ccfg);
+        nearCfg = new VisorCacheNearConfiguration(ccfg);
 
-        qryCfg = new VisorCacheQueryConfiguration().from(ccfg);
+        storeCfg = new VisorCacheStoreConfiguration(ignite, ccfg);
 
-        return this;
+        qryCfg = new VisorCacheQueryConfiguration(ccfg);
     }
 
     /**
      * @return Cache name.
      */
-    @Nullable public String name() {
+    @Nullable public String getName() {
         return name;
     }
 
     /**
      * @return Cache mode.
      */
-    public CacheMode mode() {
+    public CacheMode getMode() {
         return mode;
     }
 
     /**
      * @return Cache atomicity mode
      */
-    public CacheAtomicityMode atomicityMode() {
+    public CacheAtomicityMode getAtomicityMode() {
         return atomicityMode;
     }
 
     /**
      * @return Cache atomicity write ordering mode.
      */
-    public CacheAtomicWriteOrderMode atomicWriteOrderMode() {
+    public CacheAtomicWriteOrderMode getAtomicWriteOrderMode() {
         return atomicWriteOrderMode;
     }
 
     /**
      * @return Eager ttl flag
      */
-    public boolean eagerTtl() {
+    public boolean getEagerTtl() {
         return eagerTtl;
     }
 
     /**
      * @return Write synchronization mode.
      */
-    public CacheWriteSynchronizationMode writeSynchronizationMode() {
+    public CacheWriteSynchronizationMode getWriteSynchronizationMode() {
         return writeSynchronizationMode;
     }
 
     /**
-     * @return Swap enabled flag.
-     */
-    public boolean swapEnabled() {
-        return swapEnabled;
-    }
-
-    /**
      * @return Invalidate.
      */
-    public boolean invalidate() {
+    public boolean isInvalidate() {
         return invalidate;
     }
 
     /**
      * @return Start size.
      */
-    public int startSize() {
+    public int getStartSize() {
         return startSize;
     }
 
     /**
      * @return Off-heap max memory.
      */
-    public long offsetHeapMaxMemory() {
+    public long getOffsetHeapMaxMemory() {
         return offHeapMaxMemory;
     }
 
     /**
      * @return Max concurrent async operations
      */
-    public int maxConcurrentAsyncOperations() {
+    public int getMaxConcurrentAsyncOperations() {
         return maxConcurrentAsyncOps;
     }
 
     /**
-     * @return Memory mode.
+     * @return Cache interceptor.
      */
-    public CacheMemoryMode memoryMode() {
-        return memoryMode;
+    @Nullable public String getInterceptor() {
+        return interceptor;
     }
 
     /**
-     * @return Cache interceptor.
+     * @return Gets default lock acquisition timeout.
      */
-    @Nullable public String interceptor() {
-        return interceptor;
+    public long getDefaultLockTimeout() {
+        return dfltLockTimeout;
     }
 
     /**
      * @return Collection of type metadata.
      */
-    public Collection<VisorCacheTypeMetadata> typeMeta() {
+    public Collection<VisorCacheTypeMetadata> getTypeMeta() {
         return typeMeta;
     }
 
     /**
      * @return {@code true} if cache statistics enabled.
      */
-    public boolean statisticsEnabled() {
+    public boolean isStatisticsEnabled() {
         return statisticsEnabled;
     }
 
     /**
      * @return Whether management is enabled.
      */
-    public boolean managementEnabled() {
+    public boolean isManagementEnabled() {
         return mgmtEnabled;
     }
 
     /**
      * @return Class name of cache loader factory.
      */
-    public String loaderFactory() {
+    public String getLoaderFactory() {
         return ldrFactory;
     }
 
     /**
      * @return Class name of cache writer factory.
      */
-    public String writerFactory() {
+    public String getWriterFactory() {
         return writerFactory;
     }
 
     /**
      * @return Class name of expiry policy factory.
      */
-    public String expiryPolicyFactory() {
+    public String getExpiryPolicyFactory() {
         return expiryPlcFactory;
     }
 
     /**
      * @return Cache affinityCfg config.
      */
-    public VisorCacheAffinityConfiguration affinityConfiguration() {
+    public VisorCacheAffinityConfiguration getAffinityConfiguration() {
         return affinityCfg;
     }
 
     /**
      * @return Preload config.
      */
-    public VisorCacheRebalanceConfiguration rebalanceConfiguration() {
+    public VisorCacheRebalanceConfiguration getRebalanceConfiguration() {
         return rebalanceCfg;
     }
 
     /**
      * @return Eviction config.
      */
-    public VisorCacheEvictionConfiguration evictConfiguration() {
+    public VisorCacheEvictionConfiguration getEvictConfiguration() {
         return evictCfg;
     }
 
     /**
      * @return Near cache config.
      */
-    public VisorCacheNearConfiguration nearConfiguration() {
+    public VisorCacheNearConfiguration getNearConfiguration() {
         return nearCfg;
     }
 
     /**
-     * @return Dgc config
-     */
-    public VisorCacheDefaultConfiguration defaultConfiguration() {
-        return dfltCfg;
-    }
-
-    /**
      * @return Store config
      */
-    public VisorCacheStoreConfiguration storeConfiguration() {
+    public VisorCacheStoreConfiguration getStoreConfiguration() {
         return storeCfg;
     }
 
     /**
      * @return Cache query configuration.
      */
-    public VisorCacheQueryConfiguration queryConfiguration() {
+    public VisorCacheQueryConfiguration getQueryConfiguration() {
         return qryCfg;
     }
 
     /**
      * @return System cache state.
      */
-    public boolean system() {
+    public boolean isSystem() {
         return sys;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, name);
+        U.writeEnum(out, mode);
+        U.writeEnum(out, atomicityMode);
+        U.writeEnum(out, atomicWriteOrderMode);
+        out.writeBoolean(eagerTtl);
+        U.writeEnum(out, writeSynchronizationMode);
+        out.writeBoolean(invalidate);
+        out.writeInt(startSize);
+        out.writeLong(offHeapMaxMemory);
+        out.writeInt(maxConcurrentAsyncOps);
+        U.writeString(out, interceptor);
+        out.writeLong(dfltLockTimeout);
+        out.writeObject(affinityCfg);
+        out.writeObject(rebalanceCfg);
+        out.writeObject(evictCfg);
+        out.writeObject(nearCfg);
+        out.writeObject(storeCfg);
+        U.writeCollection(out, typeMeta);
+        out.writeBoolean(statisticsEnabled);
+        out.writeBoolean(mgmtEnabled);
+        U.writeString(out, ldrFactory);
+        U.writeString(out, writerFactory);
+        U.writeString(out, expiryPlcFactory);
+        out.writeObject(qryCfg);
+        out.writeBoolean(sys);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        name = U.readString(in);
+        mode = CacheMode.fromOrdinal(in.readByte());
+        atomicityMode = CacheAtomicityMode.fromOrdinal(in.readByte());
+        atomicWriteOrderMode = CacheAtomicWriteOrderMode.fromOrdinal(in.readByte());
+        eagerTtl = in.readBoolean();
+        writeSynchronizationMode = CacheWriteSynchronizationMode.fromOrdinal(in.readByte());
+        invalidate = in.readBoolean();
+        startSize = in.readInt();
+        offHeapMaxMemory = in.readLong();
+        maxConcurrentAsyncOps = in.readInt();
+        interceptor = U.readString(in);
+        dfltLockTimeout = in.readLong();
+        affinityCfg = (VisorCacheAffinityConfiguration)in.readObject();
+        rebalanceCfg = (VisorCacheRebalanceConfiguration)in.readObject();
+        evictCfg = (VisorCacheEvictionConfiguration)in.readObject();
+        nearCfg = (VisorCacheNearConfiguration)in.readObject();
+        storeCfg = (VisorCacheStoreConfiguration)in.readObject();
+        typeMeta = U.readCollection(in);
+        statisticsEnabled = in.readBoolean();
+        mgmtEnabled = in.readBoolean();
+        ldrFactory = U.readString(in);
+        writerFactory = U.readString(in);
+        expiryPlcFactory = U.readString(in);
+        qryCfg = (VisorCacheQueryConfiguration)in.readObject();
+        sys = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
index c872d98..c2a3c02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
@@ -67,11 +67,11 @@ public class VisorCacheConfigurationCollectorJob
      * @return Data transfer object to send it to Visor.
      */
     protected VisorCacheConfiguration config(CacheConfiguration ccfg) {
-        return new VisorCacheConfiguration().from(ignite, ccfg);
+        return new VisorCacheConfiguration(ignite, ccfg);
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheConfigurationCollectorJob.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
deleted file mode 100644
index 03b5020..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheDefaultConfiguration.java
+++ /dev/null
@@ -1,58 +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.io.Serializable;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.LessNamingBean;
-import org.apache.ignite.internal.util.typedef.internal.S;
-
-/**
- * Data transfer object for default cache configuration properties.
- */
-public class VisorCacheDefaultConfiguration implements Serializable, LessNamingBean {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Default transaction timeout. */
-    private long txLockTimeout;
-
-    /**
-     * @param ccfg Cache configuration.
-     * @return Data transfer object for default cache configuration properties.
-     */
-    public static VisorCacheDefaultConfiguration from(CacheConfiguration ccfg) {
-        VisorCacheDefaultConfiguration cfg = new VisorCacheDefaultConfiguration();
-
-        cfg.txLockTimeout = ccfg.getDefaultLockTimeout();
-
-        return cfg;
-    }
-
-    /**
-     * @return Default transaction timeout.
-     */
-    public long txLockTimeout() {
-        return txLockTimeout;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheDefaultConfiguration.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
index 05d5c38..315bdcf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheEvictionConfiguration.java
@@ -17,11 +17,14 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.cache.eviction.EvictionPolicy;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
@@ -30,7 +33,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.evictionPolic
 /**
  * Data transfer object for eviction configuration properties.
  */
-public class VisorCacheEvictionConfiguration implements Serializable, LessNamingBean {
+public class VisorCacheEvictionConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -59,83 +62,110 @@ public class VisorCacheEvictionConfiguration implements Serializable, LessNaming
     private float maxOverflowRatio;
 
     /**
-     * @param ccfg Cache configuration.
-     * @return Data transfer object for eviction configuration properties.
+     * Default constructor.
      */
-    public static VisorCacheEvictionConfiguration from(CacheConfiguration ccfg) {
-        VisorCacheEvictionConfiguration cfg = new VisorCacheEvictionConfiguration();
-
-        final EvictionPolicy plc = ccfg.getEvictionPolicy();
-
-        cfg.plc = compactClass(plc);
-        cfg.plcMaxSize = evictionPolicyMaxSize(plc);
-        cfg.filter = compactClass(ccfg.getEvictionFilter());
-        cfg.syncConcurrencyLvl = ccfg.getEvictSynchronizedConcurrencyLevel();
-        cfg.syncTimeout = ccfg.getEvictSynchronizedTimeout();
-        cfg.syncKeyBufSize = ccfg.getEvictSynchronizedKeyBufferSize();
-        cfg.evictSynchronized = ccfg.isEvictSynchronized();
-        cfg.maxOverflowRatio = ccfg.getEvictMaxOverflowRatio();
+    public VisorCacheEvictionConfiguration() {
+        // No-op.
+    }
 
-        return cfg;
+    /**
+     * Create data transfer object for eviction configuration properties.
+     * @param ccfg Cache configuration.
+     */
+    public VisorCacheEvictionConfiguration(CacheConfiguration ccfg) {
+        final EvictionPolicy evictionPlc = ccfg.getEvictionPolicy();
+
+        plc = compactClass(evictionPlc);
+        plcMaxSize = evictionPolicyMaxSize(evictionPlc);
+        filter = compactClass(ccfg.getEvictionFilter());
+        syncConcurrencyLvl = ccfg.getEvictSynchronizedConcurrencyLevel();
+        syncTimeout = ccfg.getEvictSynchronizedTimeout();
+        syncKeyBufSize = ccfg.getEvictSynchronizedKeyBufferSize();
+        evictSynchronized = ccfg.isEvictSynchronized();
+        maxOverflowRatio = ccfg.getEvictMaxOverflowRatio();
     }
 
     /**
      * @return Eviction policy.
      */
-    @Nullable public String policy() {
+    @Nullable public String getPolicy() {
         return plc;
     }
 
     /**
      * @return Cache eviction policy max size.
      */
-    @Nullable public Integer policyMaxSize() {
+    @Nullable public Integer getPolicyMaxSize() {
         return plcMaxSize;
     }
 
     /**
      * @return Eviction filter to specify which entries should not be evicted.
      */
-    @Nullable public String filter() {
+    @Nullable public String getFilter() {
         return filter;
     }
 
     /**
      * @return synchronized eviction concurrency level.
      */
-    public int synchronizedConcurrencyLevel() {
+    public int getSynchronizedConcurrencyLevel() {
         return syncConcurrencyLvl;
     }
 
     /**
      * @return synchronized eviction timeout.
      */
-    public long synchronizedTimeout() {
+    public long getSynchronizedTimeout() {
         return syncTimeout;
     }
 
     /**
      * @return Synchronized key buffer size.
      */
-    public int synchronizedKeyBufferSize() {
+    public int getSynchronizedKeyBufferSize() {
         return syncKeyBufSize;
     }
 
     /**
      * @return Synchronous evicts flag.
      */
-    public boolean evictSynchronized() {
+    public boolean isEvictSynchronized() {
         return evictSynchronized;
     }
 
     /**
      * @return Eviction max overflow ratio.
      */
-    public float maxOverflowRatio() {
+    public float getMaxOverflowRatio() {
         return maxOverflowRatio;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, plc);
+        out.writeObject(plcMaxSize);
+        U.writeString(out, filter);
+        out.writeInt(syncConcurrencyLvl);
+        out.writeLong(syncTimeout);
+        out.writeInt(syncKeyBufSize);
+        out.writeBoolean(evictSynchronized);
+        out.writeFloat(maxOverflowRatio);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        plc = U.readString(in);
+        plcMaxSize = (Integer)in.readObject();
+        filter = U.readString(in);
+        syncConcurrencyLvl = in.readInt();
+        syncTimeout = in.readLong();
+        syncKeyBufSize = in.readInt();
+        evictSynchronized = in.readBoolean();
+        maxOverflowRatio = in.readFloat();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheEvictionConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTask.java
index 212aaa9..65e1e9c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTask.java
@@ -37,18 +37,18 @@ import org.apache.ignite.internal.visor.VisorOneNodeTask;
  */
 @GridInternal
 public class VisorCacheLoadTask extends
-    VisorOneNodeTask<GridTuple3<Set<String>, Long, Object[]>, Map<String, Integer>> {
+    VisorOneNodeTask<VisorCacheLoadTaskArg, Map<String, Integer>> {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Override protected VisorCachesLoadJob job(GridTuple3<Set<String>, Long, Object[]> arg) {
+    @Override protected VisorCachesLoadJob job(VisorCacheLoadTaskArg arg) {
         return new VisorCachesLoadJob(arg, debug);
     }
 
     /** Job that load caches. */
     private static class VisorCachesLoadJob extends
-        VisorJob<GridTuple3<Set<String>, Long, Object[]>, Map<String, Integer>> {
+        VisorJob<VisorCacheLoadTaskArg, Map<String, Integer>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -56,18 +56,17 @@ public class VisorCacheLoadTask extends
          * @param arg Cache names, ttl and loader arguments.
          * @param debug Debug flag.
          */
-        private VisorCachesLoadJob(GridTuple3<Set<String>, Long, Object[]> arg, boolean debug) {
+        private VisorCachesLoadJob(VisorCacheLoadTaskArg arg, boolean debug) {
             super(arg, debug);
         }
 
         /** {@inheritDoc} */
-        @Override protected Map<String, Integer> run(GridTuple3<Set<String>, Long, Object[]> arg) {
-            Set<String> cacheNames = arg.get1();
-            Long ttl = arg.get2();
-            Object[] ldrArgs = arg.get3();
+        @Override protected Map<String, Integer> run(VisorCacheLoadTaskArg arg) {
+            Set<String> cacheNames = arg.getCacheNames();
+            long ttl = arg.getTtl();
+            Object[] ldrArgs = arg.getLdrArgs();
 
             assert cacheNames != null && !cacheNames.isEmpty();
-            assert ttl != null;
 
             Map<String, Integer> res = U.newHashMap(cacheNames.size());
 
@@ -96,4 +95,4 @@ public class VisorCacheLoadTask extends
             return S.toString(VisorCachesLoadJob.class, this);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTaskArg.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTaskArg.java
new file mode 100644
index 0000000..b210dd0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheLoadTaskArg.java
@@ -0,0 +1,101 @@
+/*
+ * 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.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Set;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * Argument for task returns cache load results.
+ */
+public class VisorCacheLoadTaskArg extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Cache names to load data. */
+    private Set<String> cacheNames;
+
+    /** Duration a Cache Entry should exist be before it expires after being modified. */
+    private long ttl;
+
+    /** Optional user arguments to be passed into CacheStore.loadCache(IgniteBiInClosure, Object...) method. */
+    private Object[] ldrArgs;
+
+    /**
+     * Default constructor.
+     */
+    public VisorCacheLoadTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * @param cacheNames Cache names to load data.
+     * @param ttl Duration a Cache Entry should exist be before it expires after being modified.
+     * @param ldrArgs Optional user arguments to be passed into CacheStore.loadCache(IgniteBiInClosure, Object...) method.
+     */
+    public VisorCacheLoadTaskArg(Set<String> cacheNames, long ttl, Object[] ldrArgs) {
+        this.cacheNames = cacheNames;
+        this.ttl = ttl;
+        this.ldrArgs = ldrArgs;
+    }
+
+    /**
+     * @return Cache names to load data.
+     */
+    public Set<String> getCacheNames() {
+        return cacheNames;
+    }
+
+    /**
+     * @return Duration a Cache Entry should exist be before it expires after being modified.
+     */
+    public long getTtl() {
+        return ttl;
+    }
+
+    /**
+     * @return Optional user arguments to be passed into CacheStore.loadCache(IgniteBiInClosure, Object...) method.
+     */
+    public Object[] getLdrArgs() {
+        return ldrArgs;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeCollection(out, cacheNames);
+        out.writeLong(ttl);
+        U.writeArray(out, ldrArgs);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        cacheNames = U.readSet(in);
+        ttl = in.readLong();
+        ldrArgs = U.readArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheLoadTaskArg.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetadataTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetadataTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetadataTask.java
index 6ba783c..598c8cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetadataTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetadataTask.java
@@ -34,7 +34,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.escapeName;
  * Task to get cache SQL metadata.
  */
 @GridInternal
-public class VisorCacheMetadataTask extends VisorOneNodeTask<String, GridCacheSqlMetadata> {
+public class VisorCacheMetadataTask extends VisorOneNodeTask<String, VisorCacheSqlMetadata> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -46,7 +46,7 @@ public class VisorCacheMetadataTask extends VisorOneNodeTask<String, GridCacheSq
     /**
      * Job to get cache SQL metadata.
      */
-    private static class VisorCacheMetadataJob extends VisorJob<String, GridCacheSqlMetadata> {
+    private static class VisorCacheMetadataJob extends VisorJob<String, VisorCacheSqlMetadata> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -59,12 +59,18 @@ public class VisorCacheMetadataTask extends VisorOneNodeTask<String, GridCacheSq
         }
 
         /** {@inheritDoc} */
-        @Override protected GridCacheSqlMetadata run(String cacheName) {
+        @Override protected VisorCacheSqlMetadata run(String cacheName) {
             try {
                 IgniteInternalCache<Object, Object> cache = ignite.cachex(cacheName);
 
-                if (cache != null)
-                    return F.first(cache.context().queries().sqlMetadata());
+                if (cache != null) {
+                    GridCacheSqlMetadata meta = F.first(cache.context().queries().sqlMetadata());
+
+                    if (meta != null)
+                        return new VisorCacheSqlMetadata(meta);
+
+                    return null;
+                }
 
                 throw new IgniteException("Cache not found: " + escapeName(cacheName));
             }
@@ -78,4 +84,4 @@ public class VisorCacheMetadataTask extends VisorOneNodeTask<String, GridCacheSq
             return S.toString(VisorCacheMetadataJob.class, this);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/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 507aacd..fba4b4e 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
@@ -17,25 +17,28 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.IgniteCache;
 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;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data transfer object for {@link CacheMetrics}.
  */
-public class VisorCacheMetrics implements Serializable, LessNamingBean {
+public class VisorCacheMetrics extends VisorDataTransferObject {
     /** */
-    private static final float MICROSECONDS_IN_SECOND = 1_000_000;
+    private static final long serialVersionUID = 0L;
 
     /** */
-    private static final long serialVersionUID = 0L;
+    private static final float MICROSECONDS_IN_SECOND = 1_000_000;
 
     /** Cache name. */
     private String name;
@@ -161,7 +164,7 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
     private long offHeapAllocatedSize;
 
     /** Number of cache entries stored in off-heap memory. */
-    private long offHeapEntriesCount;
+    private long offHeapEntriesCnt;
 
     /**
      * Calculate rate of metric per second.
@@ -174,11 +177,19 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
     }
 
     /**
+     * Default constructor.
+     */
+    public VisorCacheMetrics() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object for given cache metrics.
+     *
      * @param ignite Ignite.
      * @param cacheName Cache name.
-     * @return Data transfer object for given cache metrics.
      */
-    public VisorCacheMetrics from(IgniteEx ignite, String cacheName) {
+    public VisorCacheMetrics(IgniteEx ignite, String cacheName) {
         GridCacheProcessor cacheProcessor = ignite.context().cache();
 
         IgniteCache<Object, Object> c = cacheProcessor.jcache(cacheName);
@@ -217,7 +228,7 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
         commitsPerSec = perSecond(m.getAverageTxCommitTime());
         rollbacksPerSec = perSecond(m.getAverageTxRollbackTime());
 
-        qryMetrics = VisorCacheQueryMetrics.from(c.queryMetrics());
+        qryMetrics = new VisorCacheQueryMetrics(c.queryMetrics());
 
         dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         txThreadMapSize = m.getTxThreadMapSize();
@@ -238,15 +249,13 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
         GridCacheAdapter<Object, Object> ca = cacheProcessor.internalCache(cacheName);
 
         offHeapAllocatedSize = ca.offHeapAllocatedSize();
-        offHeapEntriesCount = ca.offHeapEntriesCount();
-
-        return this;
+        offHeapEntriesCnt = ca.offHeapEntriesCount();
     }
 
     /**
      * @return Cache name.
      */
-    public String name() {
+    public String getName() {
         return name;
     }
 
@@ -255,287 +264,287 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
      *
      * @param name New value for cache name.
      */
-    public void name(String name) {
+    public void setName(String name) {
         this.name = name;
     }
 
     /**
      * @return Cache mode.
      */
-    public CacheMode mode() {
+    public CacheMode getMode() {
         return mode;
     }
 
     /**
      * @return Cache system state.
      */
-    public boolean system() {
+    public boolean isSystem() {
         return sys;
     }
 
     /**
      * @return Total number of reads of the owning entity (either cache or entry).
      */
-    public long reads() {
+    public long getReads() {
         return reads;
     }
 
     /**
      * @return The mean time to execute gets
      */
-    public float avgReadTime() {
+    public float getAvgReadTime() {
         return avgReadTime;
     }
 
     /**
      * @return Total number of writes of the owning entity (either cache or entry).
      */
-    public long writes() {
+    public long getWrites() {
         return writes;
     }
 
     /**
      * @return Total number of hits for the owning entity (either cache or entry).
      */
-    public long hits() {
+    public long getHits() {
         return hits;
     }
 
     /**
      * @return Total number of misses for the owning entity (either cache or entry).
      */
-    public long misses() {
+    public long getMisses() {
         return misses;
     }
 
     /**
      * @return Total number of transaction commits.
      */
-    public long txCommits() {
+    public long getTxCommits() {
         return txCommits;
     }
 
     /**
      * @return avgTxCommitTime
      */
-    public float avgTxCommitTime() {
+    public float getAvgTxCommitTime() {
         return avgTxCommitTime;
     }
 
     /**
      * @return The mean time to execute tx rollbacks.
      */
-    public float avgTxRollbackTime() {
+    public float getAvgTxRollbackTime() {
         return avgTxRollbackTime;
     }
 
     /**
      * @return The total number of puts to the cache.
      */
-    public long puts() {
+    public long getPuts() {
         return puts;
     }
 
     /**
      * @return The mean time to execute puts.
      */
-    public float avgPutTime() {
+    public float getAvgPutTime() {
         return avgPutTime;
     }
 
     /**
      * @return The total number of removals from the cache.
      */
-    public long removals() {
+    public long getRemovals() {
         return removals;
     }
 
     /**
      * @return The mean time to execute removes.
      */
-    public float avgRemovalTime() {
+    public float getAvgRemovalTime() {
         return avgRemovalTime;
     }
 
     /**
      * @return The total number of evictions from the cache.
      */
-    public long evictions() {
+    public long getEvictions() {
         return evictions;
     }
 
     /**
      * @return Total number of transaction rollbacks.
      */
-    public long txRollbacks() {
+    public long getTxRollbacks() {
         return txRollbacks;
     }
 
     /**
      * @return Reads per second.
      */
-    public int readsPerSecond() {
+    public int getReadsPerSecond() {
         return readsPerSec;
     }
 
     /**
      * @return Puts per second.
      */
-    public int putsPerSecond() {
+    public int getPutsPerSecond() {
         return putsPerSec;
     }
 
     /**
      * @return Removes per second.
      */
-    public int removalsPerSecond() {
+    public int getRemovalsPerSecond() {
         return removalsPerSec;
     }
 
     /**
      * @return Commits per second.
      */
-    public int commitsPerSecond() {
+    public int getCommitsPerSecond() {
         return commitsPerSec;
     }
 
     /**
      * @return Rollbacks per second.
      */
-    public int rollbacksPerSecond() {
+    public int getRollbacksPerSecond() {
         return rollbacksPerSec;
     }
 
     /**
      * @return Number of non-{@code null} values in the cache.
      */
-    public int size() {
+    public int getSize() {
         return size;
     }
 
     /**
      * @return Gets number of keys in the cache, possibly with {@code null} values.
      */
-    public int keySize() {
+    public int getKeySize() {
         return keySize;
     }
 
     /**
      * @return Gets query metrics for cache.
      */
-    public VisorCacheQueryMetrics queryMetrics() {
+    public VisorCacheQueryMetrics getQueryMetrics() {
         return qryMetrics;
     }
 
     /**
      * @return Current size of evict queue used to batch up evictions.
      */
-    public int dhtEvictQueueCurrentSize() {
+    public int getDhtEvictQueueCurrentSize() {
         return dhtEvictQueueCurrSize;
     }
 
     /**
      * @return Gets transaction per-thread map size.
      */
-    public int txThreadMapSize() {
+    public int GetTxThreadMapSize() {
         return txThreadMapSize;
     }
 
     /**
      * @return Transaction per-Xid map size.
      */
-    public int txXidMapSize() {
+    public int getTxXidMapSize() {
         return txXidMapSize;
     }
 
     /**
      * @return Committed transaction queue size.
      */
-    public int txCommitQueueSize() {
+    public int getTxCommitQueueSize() {
         return txCommitQueueSize;
     }
 
     /**
      * @return Prepared transaction queue size.
      */
-    public int txPrepareQueueSize() {
+    public int getTxPrepareQueueSize() {
         return txPrepareQueueSize;
     }
 
     /**
      * @return Start version counts map size.
      */
-    public int txStartVersionCountsSize() {
+    public int getTxStartVersionCountsSize() {
         return txStartVerCountsSize;
     }
 
     /**
      * @return Number of cached committed transaction IDs.
      */
-    public int txCommittedVersionsSize() {
+    public int getTxCommittedVersionsSize() {
         return txCommittedVersionsSize;
     }
 
     /**
      * @return Number of cached rolled back transaction IDs.
      */
-    public int txRolledbackVersionsSize() {
+    public int getTxRolledbackVersionsSize() {
         return txRolledbackVersionsSize;
     }
 
     /**
      * @return DHT thread map size
      */
-    public int txDhtThreadMapSize() {
+    public int getTxDhtThreadMapSize() {
         return txDhtThreadMapSize;
     }
 
     /**
      * @return Transaction DHT per-Xid map size.
      */
-    public int txDhtXidMapSize() {
+    public int getTxDhtXidMapSize() {
         return txDhtXidMapSize;
     }
 
     /**
      * @return Committed DHT transaction queue size.
      */
-    public int txDhtCommitQueueSize() {
+    public int getTxDhtCommitQueueSize() {
         return txDhtCommitQueueSize;
     }
 
     /**
      * @return Prepared DHT transaction queue size.
      */
-    public int txDhtPrepareQueueSize() {
+    public int getTxDhtPrepareQueueSize() {
         return txDhtPrepareQueueSize;
     }
 
     /**
      * @return DHT start version counts map size.
      */
-    public int txDhtStartVersionCountsSize() {
+    public int getTxDhtStartVersionCountsSize() {
         return txDhtStartVerCountsSize;
     }
 
     /**
      * @return Number of cached committed DHT transaction IDs.
      */
-    public int txDhtCommittedVersionsSize() {
+    public int getTxDhtCommittedVersionsSize() {
         return txDhtCommittedVersionsSize;
     }
 
     /**
      * @return Number of cached rolled back DHT transaction IDs.
      */
-    public int txDhtRolledbackVersionsSize() {
+    public int getTxDhtRolledbackVersionsSize() {
         return txDhtRolledbackVersionsSize;
     }
 
     /**
      * @return Memory size allocated in off-heap.
      */
-    public long offHeapAllocatedSize() {
+    public long getOffHeapAllocatedSize() {
         return offHeapAllocatedSize;
     }
 
@@ -543,7 +552,99 @@ public class VisorCacheMetrics implements Serializable, LessNamingBean {
      * @return Number of cache entries stored in off-heap memory.
      */
     public long offHeapEntriesCount() {
-        return offHeapEntriesCount;
+        return offHeapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, name);
+        U.writeEnum(out, mode);
+        out.writeBoolean(sys);
+        out.writeInt(size);
+        out.writeInt(keySize);
+        out.writeLong(reads);
+        out.writeFloat(avgReadTime);
+        out.writeLong(writes);
+        out.writeLong(hits);
+        out.writeLong(misses);
+        out.writeLong(txCommits);
+        out.writeFloat(avgTxCommitTime);
+        out.writeLong(txRollbacks);
+        out.writeFloat(avgTxRollbackTime);
+        out.writeLong(puts);
+        out.writeFloat(avgPutTime);
+        out.writeLong(removals);
+        out.writeFloat(avgRemovalTime);
+        out.writeLong(evictions);
+        out.writeInt(readsPerSec);
+        out.writeInt(putsPerSec);
+        out.writeInt(removalsPerSec);
+        out.writeInt(commitsPerSec);
+        out.writeInt(rollbacksPerSec);
+        out.writeInt(dhtEvictQueueCurrSize);
+        out.writeInt(txThreadMapSize);
+        out.writeInt(txXidMapSize);
+        out.writeInt(txCommitQueueSize);
+        out.writeInt(txPrepareQueueSize);
+        out.writeInt(txStartVerCountsSize);
+        out.writeInt(txCommittedVersionsSize);
+        out.writeInt(txRolledbackVersionsSize);
+        out.writeInt(txDhtThreadMapSize);
+        out.writeInt(txDhtXidMapSize);
+        out.writeInt(txDhtCommitQueueSize);
+        out.writeInt(txDhtPrepareQueueSize);
+        out.writeInt(txDhtStartVerCountsSize);
+        out.writeInt(txDhtCommittedVersionsSize);
+        out.writeInt(txDhtRolledbackVersionsSize);
+        out.writeLong(offHeapAllocatedSize);
+        out.writeLong(offHeapEntriesCnt);
+        out.writeObject(qryMetrics);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        name = U.readString(in);
+        mode = CacheMode.fromOrdinal(in.readByte());
+        sys = in.readBoolean();
+        size = in.readInt();
+        keySize = in.readInt();
+        reads = in.readLong();
+        avgReadTime = in.readFloat();
+        writes = in.readLong();
+        hits = in.readLong();
+        misses = in.readLong();
+        txCommits = in.readLong();
+        avgTxCommitTime = in.readFloat();
+        txRollbacks = in.readLong();
+        avgTxRollbackTime = in.readFloat();
+        puts = in.readLong();
+        avgPutTime = in.readFloat();
+        removals = in.readLong();
+        avgRemovalTime = in.readFloat();
+        evictions = in.readLong();
+        readsPerSec = in.readInt();
+        putsPerSec = in.readInt();
+        removalsPerSec = in.readInt();
+        commitsPerSec = in.readInt();
+        rollbacksPerSec = in.readInt();
+        dhtEvictQueueCurrSize = in.readInt();
+        txThreadMapSize = in.readInt();
+        txXidMapSize = in.readInt();
+        txCommitQueueSize = in.readInt();
+        txPrepareQueueSize = in.readInt();
+        txStartVerCountsSize = in.readInt();
+        txCommittedVersionsSize = in.readInt();
+        txRolledbackVersionsSize = in.readInt();
+        txDhtThreadMapSize = in.readInt();
+        txDhtXidMapSize = in.readInt();
+        txDhtCommitQueueSize = in.readInt();
+        txDhtPrepareQueueSize = in.readInt();
+        txDhtStartVerCountsSize = in.readInt();
+        txDhtCommittedVersionsSize = in.readInt();
+        txDhtRolledbackVersionsSize = in.readInt();
+        offHeapAllocatedSize = in.readLong();
+        offHeapEntriesCnt = in.readLong();
+        qryMetrics = (VisorCacheQueryMetrics)in.readObject();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/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 4fd85de..a2764ec 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
@@ -36,13 +36,13 @@ import org.jetbrains.annotations.Nullable;
  * Task that collect cache metrics from all nodes.
  */
 @GridInternal
-public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiTuple<Boolean, Collection<String>>,
+public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<VisorCacheMetricsCollectorTaskArg,
     Iterable<VisorCacheAggregatedMetrics>, Collection<VisorCacheMetrics>> {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Override protected VisorCacheMetricsCollectorJob job(IgniteBiTuple<Boolean, Collection<String>> arg) {
+    @Override protected VisorCacheMetricsCollectorJob job(VisorCacheMetricsCollectorTaskArg arg) {
         return new VisorCacheMetricsCollectorJob(arg, debug);
     }
 
@@ -55,15 +55,15 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
                 Collection<VisorCacheMetrics> cms = res.getData();
 
                 for (VisorCacheMetrics cm : cms) {
-                    VisorCacheAggregatedMetrics am = grpAggrMetrics.get(cm.name());
+                    VisorCacheAggregatedMetrics am = grpAggrMetrics.get(cm.getName());
 
                     if (am == null) {
-                        am = VisorCacheAggregatedMetrics.from(cm);
+                        am = new VisorCacheAggregatedMetrics(cm);
 
-                        grpAggrMetrics.put(cm.name(), am);
+                        grpAggrMetrics.put(cm.getName(), am);
                     }
 
-                    am.metrics().put(res.getNode().id(), cm);
+                    am.getMetrics().put(res.getNode().id(), cm);
                 }
             }
         }
@@ -76,7 +76,7 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
      * Job that collect cache metrics from node.
      */
     private static class VisorCacheMetricsCollectorJob
-        extends VisorJob<IgniteBiTuple<Boolean, Collection<String>>, Collection<VisorCacheMetrics>> {
+        extends VisorJob<VisorCacheMetricsCollectorTaskArg, Collection<VisorCacheMetrics>> {
 
         /** */
         private static final long serialVersionUID = 0L;
@@ -87,19 +87,17 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
          * @param arg Whether to collect metrics for all caches or for specified cache name only.
          * @param debug Debug flag.
          */
-        private VisorCacheMetricsCollectorJob(IgniteBiTuple<Boolean, Collection<String>> arg, boolean debug) {
+        private VisorCacheMetricsCollectorJob(VisorCacheMetricsCollectorTaskArg arg, boolean debug) {
             super(arg, debug);
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<VisorCacheMetrics> run(final IgniteBiTuple<Boolean, Collection<String>> arg) {
+        @Override protected Collection<VisorCacheMetrics> run(final VisorCacheMetricsCollectorTaskArg arg) {
             assert arg != null;
 
-            Boolean showSysCaches = arg.get1();
+            Boolean showSysCaches = arg.isShowSystemCaches();
 
-            assert showSysCaches != null;
-
-            Collection<String> cacheNames = arg.get2();
+            Collection<String> cacheNames = arg.getCacheNames();
 
             assert cacheNames != null;
 
@@ -115,9 +113,9 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
                 if (ca.context().started()) {
                     String cacheName = ca.getName();
 
-                    VisorCacheMetrics cm = new VisorCacheMetrics().from(ignite, cacheName);
+                    VisorCacheMetrics cm = new VisorCacheMetrics(ignite, cacheName);
 
-                    if ((allCaches || cacheNames.contains(cacheName)) && (showSysCaches || !cm.system()))
+                    if ((allCaches || cacheNames.contains(cacheName)) && (showSysCaches || !cm.isSystem()))
                         res.add(cm);
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTaskArg.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTaskArg.java
new file mode 100644
index 0000000..f0fa60a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTaskArg.java
@@ -0,0 +1,88 @@
+/*
+ * 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.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
+
+/**
+ * Cache start arguments.
+ */
+@SuppressWarnings("PublicInnerClass")
+public class VisorCacheMetricsCollectorTaskArg extends VisorDataTransferObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Collect metrics for system caches. */
+    private boolean showSysCaches;
+
+    /** Cache names to collect metrics. */
+    private Collection<String> cacheNames;
+
+    /**
+     * Default constructor.
+     */
+    public VisorCacheMetricsCollectorTaskArg() {
+        // No-op.
+    }
+
+    /**
+     * @param showSysCaches Collect metrics for system caches.
+     * @param cacheNames Cache names to collect metrics.
+     */
+    public VisorCacheMetricsCollectorTaskArg(boolean showSysCaches, Collection<String> cacheNames) {
+        this.showSysCaches = showSysCaches;
+        this.cacheNames = cacheNames;
+    }
+
+    /**
+     * @return Collect metrics for system caches
+     */
+    public boolean isShowSystemCaches() {
+        return showSysCaches;
+    }
+
+    /**
+     * @return Cache names to collect metrics
+     */
+    public Collection<String> getCacheNames() {
+        return cacheNames;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeBoolean(showSysCaches);
+        U.writeCollection(out, cacheNames);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        showSysCaches = in.readBoolean();
+        cacheNames = U.readList(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheMetricsCollectorTaskArg.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
index 8771da4..4ec75c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheNearConfiguration.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
-import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
@@ -31,7 +34,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.evictionPolic
 /**
  * Data transfer object for near cache configuration properties.
  */
-public class VisorCacheNearConfiguration implements Serializable, LessNamingBean {
+public class VisorCacheNearConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -48,54 +51,74 @@ public class VisorCacheNearConfiguration implements Serializable, LessNamingBean
     private Integer nearEvictMaxSize;
 
     /**
-     * @param ccfg Cache configuration.
-     * @return Data transfer object for near cache configuration properties.
+     * Default constructor.
      */
-    public static VisorCacheNearConfiguration from(CacheConfiguration ccfg) {
-        VisorCacheNearConfiguration cfg = new VisorCacheNearConfiguration();
+    public VisorCacheNearConfiguration() {
+        // No-op.
+    }
 
-        cfg.nearEnabled = GridCacheUtils.isNearEnabled(ccfg);
+    /**
+     * Create data transfer object for near cache configuration properties.
+     *
+     * @param ccfg Cache configuration.
+     */
+    public VisorCacheNearConfiguration(CacheConfiguration ccfg) {
+        nearEnabled = GridCacheUtils.isNearEnabled(ccfg);
 
-        if (cfg.nearEnabled) {
+        if (nearEnabled) {
             NearCacheConfiguration nccfg = ccfg.getNearConfiguration();
 
-            cfg.nearStartSize = nccfg.getNearStartSize();
-            cfg.nearEvictPlc = compactClass(nccfg.getNearEvictionPolicy());
-            cfg.nearEvictMaxSize = evictionPolicyMaxSize(nccfg.getNearEvictionPolicy());
+            nearStartSize = nccfg.getNearStartSize();
+            nearEvictPlc = compactClass(nccfg.getNearEvictionPolicy());
+            nearEvictMaxSize = evictionPolicyMaxSize(nccfg.getNearEvictionPolicy());
         }
-
-        return cfg;
     }
 
     /**
      * @return {@code true} if near cache enabled.
      */
-    public boolean nearEnabled() {
+    public boolean isNearEnabled() {
         return nearEnabled;
     }
 
     /**
      * @return Near cache start size.
      */
-    public int nearStartSize() {
+    public int getNearStartSize() {
         return nearStartSize;
     }
 
     /**
      * @return Near cache eviction policy.
      */
-    @Nullable public String nearEvictPolicy() {
+    @Nullable public String getNearEvictPolicy() {
         return nearEvictPlc;
     }
 
     /**
      * @return Near cache eviction policy max size.
      */
-    @Nullable public Integer nearEvictMaxSize() {
+    @Nullable public Integer getNearEvictMaxSize() {
         return nearEvictMaxSize;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeBoolean(nearEnabled);
+        out.writeInt(nearStartSize);
+        U.writeString(out, nearEvictPlc);
+        out.writeObject(nearEvictMaxSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        nearEnabled = in.readBoolean();
+        nearStartSize = in.readInt();
+        nearEvictPlc = U.readString(in);
+        nearEvictMaxSize = (Integer)in.readObject();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheNearConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
index a6c0839..87cac1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartition.java
@@ -17,70 +17,67 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
-import org.apache.ignite.internal.LessNamingBean;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data transfer object for information about keys in cache partition.
  */
-public class VisorCachePartition implements Serializable, LessNamingBean {
+public class VisorCachePartition extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** */
-    private int part;
+    private int partId;
 
     /** */
-    private int heap;
+    private long cnt;
 
-    /** */
-    private long offheap;
-
-    /** */
-    private long swap;
+    /**
+     * Default constructor.
+     */
+    public VisorCachePartition() {
+        // No-op.
+    }
 
     /**
      * Full constructor.
      *
-     * @param part Partition id.
-     * @param heap Number of keys in heap.
-     * @param offheap Number of keys in offheap.
-     * @param swap Number of keys in swap.
+     * @param partId Partition id.
+     * @param cnt Number of keys in partition.
      */
-    public VisorCachePartition(int part, int heap, long offheap, long swap) {
-        this.part = part;
-        this.heap = heap;
-        this.offheap = offheap;
-        this.swap = swap;
+    public VisorCachePartition(int partId, long cnt) {
+        this.partId = partId;
+        this.cnt = cnt;
     }
 
     /**
      * @return Partition id.
      */
-    public int partition() {
-        return part;
+    public int getPartitionId() {
+        return partId;
     }
 
     /**
-     * @return Number of keys in heap.
+     * @return Number of keys in partition.
      */
-    public int heap() {
-        return heap;
+    public long getCount() {
+        return cnt;
     }
 
-    /**
-     * @return Number of keys in offheap.
-     */
-    public long offheap() {
-        return offheap;
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeInt(partId);
+        out.writeLong(cnt);
     }
 
-    /**
-     * @return Number of keys in swap.
-     */
-    public long swap() {
-        return swap;
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        partId = in.readInt();
+        cnt = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
index af48825..1cd7a29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitions.java
@@ -17,16 +17,19 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data transfer object for information about cache partitions.
  */
-public class VisorCachePartitions implements Serializable, LessNamingBean {
+public class VisorCachePartitions extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -47,42 +50,50 @@ public class VisorCachePartitions implements Serializable, LessNamingBean {
     /**
      * Add primary partition descriptor.
      *
-     * @param part Partition id.
-     * @param heap Number of primary keys in heap.
-     * @param offheap Number of primary keys in offheap.
-     * @param swap Number of primary keys in swap.
+     * @param partId Partition id.
+     * @param cnt Number of primary keys in partition.
      */
-    public void addPrimary(int part, int heap, long offheap, long swap) {
-       primary.add(new VisorCachePartition(part, heap, offheap, swap));
+    public void addPrimary(int partId, long cnt) {
+       primary.add(new VisorCachePartition(partId, cnt));
     }
 
     /**
      * Add backup partition descriptor.
      *
-     * @param part Partition id.
-     * @param heap Number of backup keys in heap.
-     * @param offheap Number of backup keys in offheap.
-     * @param swap Number of backup keys in swap.
+     * @param partId Partition id.
+     * @param cnt Number of backup keys in partition.
      */
-    public void addBackup(int part, int heap, long offheap, long swap) {
-       backup.add(new VisorCachePartition(part, heap, offheap, swap));
+    public void addBackup(int partId, long cnt) {
+       backup.add(new VisorCachePartition(partId, cnt));
     }
 
     /**
      * @return Get list of primary partitions.
      */
-    public List<VisorCachePartition> primary() {
+    public List<VisorCachePartition> getPrimary() {
         return primary;
     }
 
     /**
      * @return Get list of backup partitions.
      */
-    public List<VisorCachePartition> backup() {
+    public List<VisorCachePartition> getBackup() {
         return backup;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeCollection(out, primary);
+        U.writeCollection(out, backup);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(ObjectInput in) throws IOException, ClassNotFoundException {
+        primary = U.readList(in);
+        backup = U.readList(in);
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCachePartitions.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9112e6/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
index 9cee477..b7ca975 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePartitionsTask.java
@@ -21,14 +21,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.configuration.CacheConfiguration;
 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.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
@@ -47,7 +45,8 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.escapeName;
  * Task that collect keys distribution in partitions.
  */
 @GridInternal
-public class VisorCachePartitionsTask extends VisorMultiNodeTask<String, Map<UUID, VisorCachePartitions>, VisorCachePartitions> {
+public class VisorCachePartitionsTask extends VisorMultiNodeTask<String,
+    Map<UUID, VisorCachePartitions>, VisorCachePartitions> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -108,8 +107,6 @@ public class VisorCachePartitionsTask extends VisorMultiNodeTask<String, Map<UUI
                 && ca.context().affinityNode();
 
             if (partitioned) {
-                GridCacheSwapManager swap = ca.context().swap();
-
                 GridDhtCacheAdapter dca = null;
 
                 if (ca instanceof GridNearCacheAdapter)
@@ -122,21 +119,16 @@ public class VisorCachePartitionsTask extends VisorMultiNodeTask<String, Map<UUI
 
                     List<GridDhtLocalPartition> locParts = top.localPartitions();
 
-                    try {
-                        for (GridDhtLocalPartition part : locParts) {
-                            int p = part.id();
+                    for (GridDhtLocalPartition part : locParts) {
+                        int p = part.id();
 
-                            int sz = part.publicSize();
+                        int sz = part.publicSize();
 
-                            // Pass -1 as topology version in order not to wait for topology version.
-                            if (part.primary(AffinityTopologyVersion.NONE))
-                                parts.addPrimary(p, sz, swap.offheapEntriesCount(p), swap.swapEntriesCount(p));
-                            else if (part.state() == GridDhtPartitionState.OWNING && part.backup(AffinityTopologyVersion.NONE))
-                                parts.addBackup(p, sz, swap.offheapEntriesCount(p), swap.swapEntriesCount(p));
-                        }
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteException("Failed to collect keys distribution in partitions", e);
+                        // Pass -1 as topology version in order not to wait for topology version.
+                        if (part.primary(AffinityTopologyVersion.NONE))
+                            parts.addPrimary(p, sz);
+                        else if (part.state() == GridDhtPartitionState.OWNING && part.backup(AffinityTopologyVersion.NONE))
+                            parts.addBackup(p, sz);
                     }
                 }
             }