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

[11/50] [abbrv] ignite git commit: IGNITE-4988 Cleanup and refactor VisorXxx tasks and DTO for ignite-2.0

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
index a6d8c4c..79b468d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorAtomicConfiguration.java
@@ -17,16 +17,19 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.AtomicConfiguration;
-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 configuration of atomic data structures.
  */
-public class VisorAtomicConfiguration implements Serializable, LessNamingBean {
+public class VisorAtomicConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -40,43 +43,59 @@ public class VisorAtomicConfiguration implements Serializable, LessNamingBean {
     private int backups;
 
     /**
+     * Default constructor.
+     */
+    public VisorAtomicConfiguration() {
+        // No-op.
+    }
+
+    /**
      * Create data transfer object for atomic configuration.
      *
      * @param src Atomic configuration.
-     * @return Data transfer object.
      */
-    public static VisorAtomicConfiguration from(AtomicConfiguration src) {
-        VisorAtomicConfiguration cfg = new VisorAtomicConfiguration();
-
-        cfg.seqReserveSize = src.getAtomicSequenceReserveSize();
-        cfg.cacheMode = src.getCacheMode();
-        cfg.backups = src.getBackups();
-
-        return cfg;
+    public VisorAtomicConfiguration(AtomicConfiguration src) {
+        seqReserveSize = src.getAtomicSequenceReserveSize();
+        cacheMode = src.getCacheMode();
+        backups = src.getBackups();
     }
 
     /**
      * @return Atomic sequence reservation size.
      */
-    public int atomicSequenceReserveSize() {
+    public int getAtomicSequenceReserveSize() {
         return seqReserveSize;
     }
 
     /**
      * @return Cache mode.
      */
-    public CacheMode cacheMode() {
+    public CacheMode getCacheMode() {
         return cacheMode;
     }
 
     /**
      * @return Number of backup nodes.
      */
-    public int backups() {
+    public int getBackups() {
         return backups;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeInt(seqReserveSize);
+        U.writeEnum(out, cacheMode);
+        out.writeInt(backups);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        seqReserveSize = in.readInt();
+        cacheMode = CacheMode.fromOrdinal(in.readByte());
+        backups = in.readInt();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorAtomicConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
index 0dae4da..a70cfdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
@@ -17,12 +17,16 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.UUID;
+import org.apache.ignite.configuration.DeploymentMode;
 import org.apache.ignite.configuration.IgniteConfiguration;
 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 java.lang.System.getProperty;
@@ -38,12 +42,11 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_UPDATE_NOTIFIER;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.boolValue;
 import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
-import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactObject;
 
 /**
  * Data transfer object for node basic configuration properties.
  */
-public class VisorBasicConfiguration implements Serializable, LessNamingBean {
+public class VisorBasicConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -63,7 +66,7 @@ public class VisorBasicConfiguration implements Serializable, LessNamingBean {
     private String marsh;
 
     /** Deployment Mode. */
-    private Object deployMode;
+    private DeploymentMode deployMode;
 
     /** Client mode flag. */
     private Boolean clientMode;
@@ -111,186 +114,240 @@ public class VisorBasicConfiguration implements Serializable, LessNamingBean {
     private boolean updateNtf;
 
     /**
+     * Default constructor.
+     */
+    public VisorBasicConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object for node basic configuration properties.
+     *
      * @param ignite Grid.
      * @param c Grid configuration.
-     * @return Data transfer object for node basic configuration properties.
      */
-    public static VisorBasicConfiguration from(IgniteEx ignite, IgniteConfiguration c) {
-        VisorBasicConfiguration cfg = new VisorBasicConfiguration();
-
-        cfg.igniteInstanceName = c.getIgniteInstanceName();
-        cfg.ggHome = getProperty(IGNITE_HOME, c.getIgniteHome());
-        cfg.locHost = getProperty(IGNITE_LOCAL_HOST, c.getLocalHost());
-        cfg.nodeId = ignite.localNode().id();
-        cfg.marsh = compactClass(c.getMarshaller());
-        cfg.deployMode = compactObject(c.getDeploymentMode());
-        cfg.clientMode = c.isClientMode();
-        cfg.daemon = boolValue(IGNITE_DAEMON, c.isDaemon());
-        cfg.jmxRemote = ignite.isJmxRemoteEnabled();
-        cfg.restart = ignite.isRestartEnabled();
-        cfg.netTimeout = c.getNetworkTimeout();
-        cfg.log = compactClass(c.getGridLogger());
-        cfg.discoStartupDelay = c.getDiscoveryStartupDelay();
-        cfg.mBeanSrv = compactClass(c.getMBeanServer());
-        cfg.noAscii = boolValue(IGNITE_NO_ASCII, false);
-        cfg.noDiscoOrder = boolValue(IGNITE_NO_DISCO_ORDER, false);
-        cfg.noShutdownHook = boolValue(IGNITE_NO_SHUTDOWN_HOOK, false);
-        cfg.progName = getProperty(IGNITE_PROG_NAME);
-        cfg.quiet = boolValue(IGNITE_QUIET, true);
-        cfg.successFile = getProperty(IGNITE_SUCCESS_FILE);
-        cfg.updateNtf = boolValue(IGNITE_UPDATE_NOTIFIER, true);
-
-        return cfg;
+    public VisorBasicConfiguration(IgniteEx ignite, IgniteConfiguration c) {
+        igniteInstanceName = c.getIgniteInstanceName();
+        ggHome = getProperty(IGNITE_HOME, c.getIgniteHome());
+        locHost = getProperty(IGNITE_LOCAL_HOST, c.getLocalHost());
+        nodeId = ignite.localNode().id();
+        marsh = compactClass(c.getMarshaller());
+        deployMode = c.getDeploymentMode();
+        clientMode = c.isClientMode();
+        daemon = boolValue(IGNITE_DAEMON, c.isDaemon());
+        jmxRemote = ignite.isJmxRemoteEnabled();
+        restart = ignite.isRestartEnabled();
+        netTimeout = c.getNetworkTimeout();
+        log = compactClass(c.getGridLogger());
+        discoStartupDelay = c.getDiscoveryStartupDelay();
+        mBeanSrv = compactClass(c.getMBeanServer());
+        noAscii = boolValue(IGNITE_NO_ASCII, false);
+        noDiscoOrder = boolValue(IGNITE_NO_DISCO_ORDER, false);
+        noShutdownHook = boolValue(IGNITE_NO_SHUTDOWN_HOOK, false);
+        progName = getProperty(IGNITE_PROG_NAME);
+        quiet = boolValue(IGNITE_QUIET, true);
+        successFile = getProperty(IGNITE_SUCCESS_FILE);
+        updateNtf = boolValue(IGNITE_UPDATE_NOTIFIER, true);
     }
 
     /**
      * @return Ignite instance name.
      */
-    @Nullable public String igniteInstanceName() {
+    @Nullable public String getIgniteInstanceName() {
         return igniteInstanceName;
     }
 
     /**
      * @return IGNITE_HOME determined at startup.
      */
-    @Nullable public String ggHome() {
+    @Nullable public String getGgHome() {
         return ggHome;
     }
 
     /**
      * @return Local host value used.
      */
-    @Nullable public String localHost() {
+    @Nullable public String getLocalHost() {
         return locHost;
     }
 
     /**
      * @return Node id.
      */
-    public UUID nodeId() {
+    public UUID getNodeId() {
         return nodeId;
     }
 
     /**
      * @return Marshaller used.
      */
-    public String marshaller() {
+    public String getMarshaller() {
         return marsh;
     }
 
     /**
      * @return Deployment Mode.
      */
-    public Object deploymentMode() {
+    public Object getDeploymentMode() {
         return deployMode;
     }
 
     /**
      * @return Client mode flag.
      */
-    public Boolean clientMode() {
+    public Boolean isClientMode() {
         return clientMode;
     }
 
     /**
      * @return Whether this node daemon or not.
      */
-    public boolean daemon() {
+    public boolean isDaemon() {
         return daemon;
     }
 
     /**
      * @return Whether remote JMX is enabled.
      */
-    public boolean jmxRemote() {
+    public boolean isJmxRemote() {
         return jmxRemote;
     }
 
     /**
      * @return Is node restart enabled.
      */
-    public boolean restart() {
+    public boolean isRestart() {
         return restart;
     }
 
     /**
      * @return Network timeout.
      */
-    public long networkTimeout() {
+    public long getNetworkTimeout() {
         return netTimeout;
     }
 
     /**
      * @return Logger used on node.
      */
-    public String logger() {
+    public String getLogger() {
         return log;
     }
 
     /**
      * @return Discovery startup delay.
      */
-    public long discoStartupDelay() {
+    public long getDiscoStartupDelay() {
         return discoStartupDelay;
     }
 
     /**
      * @return MBean server name
      */
-    @Nullable public String mBeanServer() {
+    @Nullable public String getMBeanServer() {
         return mBeanSrv;
     }
 
     /**
      * @return Whether ASCII logo is disabled.
      */
-    public boolean noAscii() {
+    public boolean isNoAscii() {
         return noAscii;
     }
 
     /**
      * @return Whether no discovery order is allowed.
      */
-    public boolean noDiscoOrder() {
+    public boolean isNoDiscoOrder() {
         return noDiscoOrder;
     }
 
     /**
      * @return Whether shutdown hook is disabled.
      */
-    public boolean noShutdownHook() {
+    public boolean isNoShutdownHook() {
         return noShutdownHook;
     }
 
     /**
      * @return Name of command line program.
      */
-    public String programName() {
+    public String getProgramName() {
         return progName;
     }
 
     /**
      * @return Whether node is in quiet mode.
      */
-    public boolean quiet() {
+    public boolean isQuiet() {
         return quiet;
     }
 
     /**
      * @return Success file name.
      */
-    public String successFile() {
+    public String getSuccessFile() {
         return successFile;
     }
 
     /**
      * @return Whether update checker is enabled.
      */
-    public boolean updateNotifier() {
+    public boolean isUpdateNotifier() {
         return updateNtf;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, igniteInstanceName);
+        U.writeString(out, ggHome);
+        U.writeString(out, locHost);
+        U.writeUuid(out, nodeId);
+        U.writeString(out, marsh);
+        U.writeEnum(out, deployMode);
+        out.writeObject(clientMode);
+        out.writeBoolean(daemon);
+        out.writeBoolean(jmxRemote);
+        out.writeBoolean(restart);
+        out.writeLong(netTimeout);
+        U.writeString(out, log);
+        out.writeLong(discoStartupDelay);
+        U.writeString(out, mBeanSrv);
+        out.writeBoolean(noAscii);
+        out.writeBoolean(noDiscoOrder);
+        out.writeBoolean(noShutdownHook);
+        U.writeString(out, progName);
+        out.writeBoolean(quiet);
+        U.writeString(out, successFile);
+        out.writeBoolean(updateNtf);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        igniteInstanceName = U.readString(in);
+        ggHome = U.readString(in);
+        locHost = U.readString(in);
+        nodeId = U.readUuid(in);
+        marsh = U.readString(in);
+        deployMode = DeploymentMode.fromOrdinal(in.readByte());
+        clientMode = (Boolean)in.readObject();
+        daemon = in.readBoolean();
+        jmxRemote = in.readBoolean();
+        restart = in.readBoolean();
+        netTimeout = in.readLong();
+        log = U.readString(in);
+        discoStartupDelay = in.readLong();
+        mBeanSrv = U.readString(in);
+        noAscii = in.readBoolean();
+        noDiscoOrder = in.readBoolean();
+        noShutdownHook = in.readBoolean();
+        progName = U.readString(in);
+        quiet = in.readBoolean();
+        successFile = U.readString(in);
+        updateNtf = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorBasicConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorExecutorServiceConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorExecutorServiceConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorExecutorServiceConfiguration.java
index df083f5..0ad9288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorExecutorServiceConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorExecutorServiceConfiguration.java
@@ -17,16 +17,18 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data transfer object for node executors configuration properties.
  */
-public class VisorExecutorServiceConfiguration implements Serializable, LessNamingBean {
+public class VisorExecutorServiceConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -45,73 +47,110 @@ public class VisorExecutorServiceConfiguration implements Serializable, LessNami
     /** Peer-to-peer pool size. */
     private int p2pPoolSz;
 
+    /** Rebalance thread pool size. */
+    private int rebalanceThreadPoolSize;
+
     /** REST requests pool size. */
     private int restPoolSz;
 
     /**
-     * @param c Grid configuration.
-     * @return Data transfer object for node executors configuration properties.
+     * Default constructor.
      */
-    public static VisorExecutorServiceConfiguration from(IgniteConfiguration c) {
-        VisorExecutorServiceConfiguration cfg = new VisorExecutorServiceConfiguration();
+    public VisorExecutorServiceConfiguration() {
+        // No-op.
+    }
 
-        cfg.pubPoolSize = c.getPublicThreadPoolSize();
-        cfg.sysPoolSz = c.getSystemThreadPoolSize();
-        cfg.mgmtPoolSize = c.getManagementThreadPoolSize();
-        cfg.p2pPoolSz = c.getPeerClassLoadingThreadPoolSize();
-        cfg.igfsPoolSize = c.getIgfsThreadPoolSize();
+    /**
+     * Create data transfer object for node executors configuration properties.
+     *
+     * @param c Grid configuration.
+     */
+    public VisorExecutorServiceConfiguration(IgniteConfiguration c) {
+        pubPoolSize = c.getPublicThreadPoolSize();
+        sysPoolSz = c.getSystemThreadPoolSize();
+        mgmtPoolSize = c.getManagementThreadPoolSize();
+        p2pPoolSz = c.getPeerClassLoadingThreadPoolSize();
+        igfsPoolSize = c.getIgfsThreadPoolSize();
+        rebalanceThreadPoolSize = c.getRebalanceThreadPoolSize();
 
         ConnectorConfiguration cc = c.getConnectorConfiguration();
 
         if (cc != null)
-            cfg.restPoolSz = cc.getThreadPoolSize();
-
-        return cfg;
+            restPoolSz = cc.getThreadPoolSize();
     }
 
     /**
      * @return Public pool size.
      */
-    public int publicThreadPoolSize() {
+    public int getPublicThreadPoolSize() {
         return pubPoolSize;
     }
 
     /**
      * @return System pool size.
      */
-    public int systemThreadPoolSize() {
+    public int getSystemThreadPoolSize() {
         return sysPoolSz;
     }
 
     /**
      * @return Management pool size.
      */
-    public int managementThreadPoolSize() {
+    public int getManagementThreadPoolSize() {
         return mgmtPoolSize;
     }
 
     /**
      * @return IGFS pool size.
      */
-    public int igfsThreadPoolSize() {
+    public int getIgfsThreadPoolSize() {
         return igfsPoolSize;
     }
 
     /**
      * @return Peer-to-peer pool size.
      */
-    public int peerClassLoadingThreadPoolSize() {
+    public int getPeerClassLoadingThreadPoolSize() {
         return p2pPoolSz;
     }
 
     /**
+     * @return Rebalance thread pool size.
+     */
+    public int getRebalanceThreadPoolSize() {
+        return rebalanceThreadPoolSize;
+    }
+
+    /**
      * @return REST requests pool size.
      */
-    public int restThreadPoolSize() {
+    public int getRestThreadPoolSize() {
         return restPoolSz;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeInt(pubPoolSize);
+        out.writeInt(sysPoolSz);
+        out.writeInt(mgmtPoolSize);
+        out.writeInt(igfsPoolSize);
+        out.writeInt(p2pPoolSz);
+        out.writeInt(rebalanceThreadPoolSize);
+        out.writeInt(restPoolSz);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        pubPoolSize = in.readInt();
+        sysPoolSz = in.readInt();
+        mgmtPoolSize = in.readInt();
+        igfsPoolSize = in.readInt();
+        p2pPoolSz = in.readInt();
+        rebalanceThreadPoolSize = in.readInt();
+        restPoolSz = in.readInt();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorExecutorServiceConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
index 0ce13df..23a74e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
@@ -17,22 +17,26 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Properties;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.IgniteConfiguration;
 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 static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactArray;
 
 /**
  * Data transfer object for node configuration data.
  */
-public class VisorGridConfiguration implements Serializable, LessNamingBean {
+public class VisorGridConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -69,8 +73,8 @@ public class VisorGridConfiguration implements Serializable, LessNamingBean {
     /** User attributes. */
     private Map<String, ?> userAttrs;
 
-    /** Igfss. */
-    private Iterable<VisorIgfsConfiguration> igfss;
+    /** IGFSs. */
+    private List<VisorIgfsConfiguration> igfss;
 
     /** Environment. */
     private Map<String, String> env;
@@ -88,155 +92,205 @@ public class VisorGridConfiguration implements Serializable, LessNamingBean {
     private VisorMemoryConfiguration memCfg;
 
     /**
+     * Default constructor.
+     */
+    public VisorGridConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object with node configuration data.
+     *
      * @param ignite Grid.
-     * @return Fill data transfer object with node configuration data.
      */
-    public VisorGridConfiguration from(IgniteEx ignite) {
+    public VisorGridConfiguration(IgniteEx ignite) {
         assert ignite != null;
 
         IgniteConfiguration c = ignite.configuration();
 
-        basic = VisorBasicConfiguration.from(ignite, c);
-        metrics = VisorMetricsConfiguration.from(c);
-        spis = VisorSpisConfiguration.from(c);
-        p2p = VisorPeerToPeerConfiguration.from(c);
-        lifecycle = VisorLifecycleConfiguration.from(c);
-        execSvc = VisorExecutorServiceConfiguration.from(c);
-        seg = VisorSegmentationConfiguration.from(c);
+        basic = new VisorBasicConfiguration(ignite, c);
+        metrics = new VisorMetricsConfiguration(c);
+        spis = new VisorSpisConfiguration(c);
+        p2p = new VisorPeerToPeerConfiguration(c);
+        lifecycle = new VisorLifecycleConfiguration(c);
+        execSvc = new VisorExecutorServiceConfiguration(c);
+        seg = new VisorSegmentationConfiguration(c);
         inclProps = compactArray(c.getIncludeProperties());
         inclEvtTypes = c.getIncludeEventTypes();
-        rest = VisorRestConfiguration.from(c);
+        rest = new VisorRestConfiguration(c);
         userAttrs = c.getUserAttributes();
         igfss = VisorIgfsConfiguration.list(c.getFileSystemConfiguration());
         env = new HashMap<>(System.getenv());
         sysProps = IgniteSystemProperties.snapshot();
-        atomic = VisorAtomicConfiguration.from(c.getAtomicConfiguration());
-        txCfg = VisorTransactionConfiguration.from(c.getTransactionConfiguration());
-        memCfg = new VisorMemoryConfiguration(c.getMemoryConfiguration());
+        atomic = new VisorAtomicConfiguration(c.getAtomicConfiguration());
+        txCfg = new VisorTransactionConfiguration(c.getTransactionConfiguration());
 
-        return this;
+        if (c.getMemoryConfiguration() != null)
+            memCfg = new VisorMemoryConfiguration(c.getMemoryConfiguration());
     }
 
     /**
      * @return Basic.
      */
-    public VisorBasicConfiguration basic() {
+    public VisorBasicConfiguration getBasic() {
         return basic;
     }
 
     /**
      * @return Metrics.
      */
-    public VisorMetricsConfiguration metrics() {
+    public VisorMetricsConfiguration getMetrics() {
         return metrics;
     }
 
     /**
      * @return SPIs.
      */
-    public VisorSpisConfiguration spis() {
+    public VisorSpisConfiguration getSpis() {
         return spis;
     }
 
     /**
      * @return P2P.
      */
-    public VisorPeerToPeerConfiguration p2p() {
+    public VisorPeerToPeerConfiguration getP2p() {
         return p2p;
     }
 
     /**
      * @return Lifecycle.
      */
-    public VisorLifecycleConfiguration lifecycle() {
+    public VisorLifecycleConfiguration getLifecycle() {
         return lifecycle;
     }
 
     /**
      * @return Executors service configuration.
      */
-    public VisorExecutorServiceConfiguration executeService() {
+    public VisorExecutorServiceConfiguration getExecutorService() {
         return execSvc;
     }
 
     /**
      * @return Segmentation.
      */
-    public VisorSegmentationConfiguration segmentation() {
+    public VisorSegmentationConfiguration getSegmentation() {
         return seg;
     }
 
     /**
      * @return Include properties.
      */
-    public String includeProperties() {
+    public String getIncludeProperties() {
         return inclProps;
     }
 
     /**
      * @return Include events types.
      */
-    public int[] includeEventTypes() {
+    public int[] getIncludeEventTypes() {
         return inclEvtTypes;
     }
 
     /**
      * @return Rest.
      */
-    public VisorRestConfiguration rest() {
+    public VisorRestConfiguration getRest() {
         return rest;
     }
 
     /**
      * @return User attributes.
      */
-    public Map<String, ?> userAttributes() {
+    public Map<String, ?> getUserAttributes() {
         return userAttrs;
     }
 
     /**
      * @return Igfss.
      */
-    public Iterable<VisorIgfsConfiguration> igfss() {
+    public List<VisorIgfsConfiguration> getIgfss() {
         return igfss;
     }
 
     /**
      * @return Environment.
      */
-    public Map<String, String> env() {
+    public Map<String, String> getEnv() {
         return env;
     }
 
     /**
      * @return System properties.
      */
-    public Properties systemProperties() {
+    public Properties getSystemProperties() {
         return sysProps;
     }
 
     /**
      * @return Configuration of atomic data structures.
      */
-    public VisorAtomicConfiguration atomic() {
+    public VisorAtomicConfiguration getAtomic() {
         return atomic;
     }
 
     /**
      * @return Transactions configuration.
      */
-    public VisorTransactionConfiguration transaction() {
+    public VisorTransactionConfiguration getTransaction() {
         return txCfg;
     }
 
     /**
      * @return Memory configuration.
      */
-    public VisorMemoryConfiguration memoryConfiguration() {
+    public VisorMemoryConfiguration getMemoryConfiguration() {
         return memCfg;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeObject(basic);
+        out.writeObject(metrics);
+        out.writeObject(spis);
+        out.writeObject(p2p);
+        out.writeObject(lifecycle);
+        out.writeObject(execSvc);
+        out.writeObject(seg);
+        U.writeString(out, inclProps);
+        out.writeObject(inclEvtTypes);
+        out.writeObject(rest);
+        U.writeMap(out, userAttrs);
+        U.writeCollection(out, igfss);
+        U.writeMap(out, env);
+        out.writeObject(sysProps);
+        out.writeObject(atomic);
+        out.writeObject(txCfg);
+        out.writeObject(memCfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        basic = (VisorBasicConfiguration)in.readObject();
+        metrics = (VisorMetricsConfiguration)in.readObject();
+        spis = (VisorSpisConfiguration)in.readObject();
+        p2p = (VisorPeerToPeerConfiguration)in.readObject();
+        lifecycle = (VisorLifecycleConfiguration)in.readObject();
+        execSvc = (VisorExecutorServiceConfiguration)in.readObject();
+        seg = (VisorSegmentationConfiguration)in.readObject();
+        inclProps = U.readString(in);
+        inclEvtTypes = (int[])in.readObject();
+        rest = (VisorRestConfiguration)in.readObject();
+        userAttrs = U.readMap(in);
+        igfss = U.readList(in);
+        env = U.readMap(in);
+        sysProps = (Properties)in.readObject();
+        atomic = (VisorAtomicConfiguration)in.readObject();
+        txCfg = (VisorTransactionConfiguration)in.readObject();
+        memCfg = (VisorMemoryConfiguration)in.readObject();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorGridConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
index 664dac1..15c1fc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
@@ -17,16 +17,19 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
-import org.apache.ignite.internal.LessNamingBean;
+import org.apache.ignite.internal.util.typedef.F;
 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;
@@ -34,7 +37,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactClass;
 /**
  * Data transfer object for IGFS configuration properties.
  */
-public class VisorIgfsConfiguration implements Serializable, LessNamingBean {
+public class VisorIgfsConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -114,44 +117,47 @@ public class VisorIgfsConfiguration implements Serializable, LessNamingBean {
     private long trashPurgeTimeout;
 
     /**
+     * Default constructor.
+     */
+    public VisorIgfsConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * Create data transfer object for IGFS configuration properties.
      * @param igfs IGFS configuration.
-     * @return Data transfer object for IGFS configuration properties.
      */
-    public static VisorIgfsConfiguration from(FileSystemConfiguration igfs) {
-        VisorIgfsConfiguration cfg = new VisorIgfsConfiguration();
-
-        cfg.name = igfs.getName();
-        cfg.metaCacheName = igfs.getMetaCacheConfiguration().getName();
-        cfg.dataCacheName = igfs.getDataCacheConfiguration().getName();
-        cfg.blockSize = igfs.getBlockSize();
-        cfg.prefetchBlocks = igfs.getPrefetchBlocks();
-        cfg.streamBufSize = igfs.getBufferSize();
-        cfg.perNodeBatchSize = igfs.getPerNodeBatchSize();
-        cfg.perNodeParallelBatchCnt = igfs.getPerNodeParallelBatchCount();
-
-        cfg.dfltMode = igfs.getDefaultMode();
-        cfg.pathModes = igfs.getPathModes();
-        cfg.dualModePutExecutorSrvc = compactClass(igfs.getDualModePutExecutorService());
-        cfg.dualModePutExecutorSrvcShutdown = igfs.getDualModePutExecutorServiceShutdown();
-        cfg.dualModeMaxPendingPutsSize = igfs.getDualModeMaxPendingPutsSize();
-        cfg.maxTaskRangeLen = igfs.getMaximumTaskRangeLength();
-        cfg.fragmentizerConcurrentFiles = igfs.getFragmentizerConcurrentFiles();
-        cfg.fragmentizerLocWritesRatio = igfs.getFragmentizerLocalWritesRatio();
-        cfg.fragmentizerEnabled = igfs.isFragmentizerEnabled();
-        cfg.fragmentizerThrottlingBlockLen = igfs.getFragmentizerThrottlingBlockLength();
-        cfg.fragmentizerThrottlingDelay = igfs.getFragmentizerThrottlingDelay();
+    public VisorIgfsConfiguration(FileSystemConfiguration igfs) {
+        name = igfs.getName();
+        metaCacheName = igfs.getMetaCacheConfiguration().getName();
+        dataCacheName = igfs.getDataCacheConfiguration().getName();
+        blockSize = igfs.getBlockSize();
+        prefetchBlocks = igfs.getPrefetchBlocks();
+        streamBufSize = igfs.getBufferSize();
+        perNodeBatchSize = igfs.getPerNodeBatchSize();
+        perNodeParallelBatchCnt = igfs.getPerNodeParallelBatchCount();
+
+        dfltMode = igfs.getDefaultMode();
+        pathModes = igfs.getPathModes();
+        dualModePutExecutorSrvc = compactClass(igfs.getDualModePutExecutorService());
+        dualModePutExecutorSrvcShutdown = igfs.getDualModePutExecutorServiceShutdown();
+        dualModeMaxPendingPutsSize = igfs.getDualModeMaxPendingPutsSize();
+        maxTaskRangeLen = igfs.getMaximumTaskRangeLength();
+        fragmentizerConcurrentFiles = igfs.getFragmentizerConcurrentFiles();
+        fragmentizerLocWritesRatio = igfs.getFragmentizerLocalWritesRatio();
+        fragmentizerEnabled = igfs.isFragmentizerEnabled();
+        fragmentizerThrottlingBlockLen = igfs.getFragmentizerThrottlingBlockLength();
+        fragmentizerThrottlingDelay = igfs.getFragmentizerThrottlingDelay();
 
         IgfsIpcEndpointConfiguration endpointCfg = igfs.getIpcEndpointConfiguration();
 
-        cfg.ipcEndpointCfg = endpointCfg != null ? endpointCfg.toString() : null;
-
-        cfg.ipcEndpointEnabled = igfs.isIpcEndpointEnabled();
-        cfg.maxSpace = igfs.getMaxSpaceSize();
-        cfg.mgmtPort = igfs.getManagementPort();
-        cfg.seqReadsBeforePrefetch = igfs.getSequentialReadsBeforePrefetch();
-        cfg.trashPurgeTimeout = igfs.getTrashPurgeTimeout();
+        ipcEndpointCfg = endpointCfg != null ? endpointCfg.toString() : null;
 
-        return cfg;
+        ipcEndpointEnabled = igfs.isIpcEndpointEnabled();
+        maxSpace = igfs.getMaxSpaceSize();
+        mgmtPort = igfs.getManagementPort();
+        seqReadsBeforePrefetch = igfs.getSequentialReadsBeforePrefetch();
+        trashPurgeTimeout = igfs.getTrashPurgeTimeout();
     }
 
     /**
@@ -160,194 +166,251 @@ public class VisorIgfsConfiguration implements Serializable, LessNamingBean {
      * @param igfss Igfs configurations.
      * @return igfs configurations properties.
      */
-    public static Iterable<VisorIgfsConfiguration> list(FileSystemConfiguration[] igfss) {
-        if (igfss == null)
-            return Collections.emptyList();
-
-        final Collection<VisorIgfsConfiguration> cfgs = new ArrayList<>(igfss.length);
+    public static List<VisorIgfsConfiguration> list(FileSystemConfiguration[] igfss) {
+        List<VisorIgfsConfiguration> res = new ArrayList<>();
 
-        for (FileSystemConfiguration igfs : igfss)
-            cfgs.add(from(igfs));
+        if (!F.isEmpty(igfss)) {
+            for (FileSystemConfiguration igfs : igfss)
+                res.add(new VisorIgfsConfiguration(igfs));
+        }
 
-        return cfgs;
+        return res;
     }
 
     /**
      * @return IGFS instance name.
      */
-    @Nullable public String name() {
+    @Nullable public String getName() {
         return name;
     }
 
     /**
      * @return Cache name to store IGFS meta information.
      */
-    @Nullable public String metaCacheName() {
+    @Nullable public String getMetaCacheName() {
         return metaCacheName;
     }
 
     /**
      * @return Cache name to store IGFS data.
      */
-    @Nullable public String dataCacheName() {
+    @Nullable public String getDataCacheName() {
         return dataCacheName;
     }
 
     /**
      * @return File's data block size.
      */
-    public int blockSize() {
+    public int getBlockSize() {
         return blockSize;
     }
 
     /**
      * @return Number of pre-fetched blocks if specific file's chunk is requested.
      */
-    public int prefetchBlocks() {
+    public int getPrefetchBlocks() {
         return prefetchBlocks;
     }
 
     /**
      * @return Read/write buffer size for IGFS stream operations in bytes.
      */
-    public int streamBufferSize() {
+    public int getStreamBufferSize() {
         return streamBufSize;
     }
 
     /**
      * @return Number of file blocks buffered on local node before sending batch to remote node.
      */
-    public int perNodeBatchSize() {
+    public int getPerNodeBatchSize() {
         return perNodeBatchSize;
     }
 
     /**
      * @return Number of batches that can be concurrently sent to remote node.
      */
-    public int perNodeParallelBatchCount() {
+    public int getPerNodeParallelBatchCount() {
         return perNodeParallelBatchCnt;
     }
 
     /**
      * @return IGFS instance mode.
      */
-    public IgfsMode defaultMode() {
+    public IgfsMode getDefaultMode() {
         return dfltMode;
     }
 
     /**
      * @return Map of paths to IGFS modes.
      */
-    @Nullable public Map<String, IgfsMode> pathModes() {
+    @Nullable public Map<String, IgfsMode> getPathModes() {
         return pathModes;
     }
 
     /**
      * @return Dual mode PUT operations executor service.
      */
-    public String dualModePutExecutorService() {
+    public String getDualModePutExecutorService() {
         return dualModePutExecutorSrvc;
     }
 
     /**
      * @return Dual mode PUT operations executor service shutdown flag.
      */
-    public boolean dualModePutExecutorServiceShutdown() {
+    public boolean getDualModePutExecutorServiceShutdown() {
         return dualModePutExecutorSrvcShutdown;
     }
 
     /**
      * @return Maximum amount of data in pending puts.
      */
-    public long dualModeMaxPendingPutsSize() {
+    public long getDualModeMaxPendingPutsSize() {
         return dualModeMaxPendingPutsSize;
     }
 
     /**
      * @return Maximum range length.
      */
-    public long maxTaskRangeLength() {
+    public long getMaxTaskRangeLength() {
         return maxTaskRangeLen;
     }
 
     /**
      * @return Fragmentizer concurrent files.
      */
-    public int fragmentizerConcurrentFiles() {
+    public int getFragmentizerConcurrentFiles() {
         return fragmentizerConcurrentFiles;
     }
 
     /**
      * @return Fragmentizer local writes ratio.
      */
-    public float fragmentizerLocalWritesRatio() {
+    public float getFragmentizerLocalWritesRatio() {
         return fragmentizerLocWritesRatio;
     }
 
     /**
      * @return Fragmentizer enabled flag.
      */
-    public boolean fragmentizerEnabled() {
+    public boolean isFragmentizerEnabled() {
         return fragmentizerEnabled;
     }
 
     /**
      * @return Fragmentizer throttling block length.
      */
-    public long fragmentizerThrottlingBlockLength() {
+    public long getFragmentizerThrottlingBlockLength() {
         return fragmentizerThrottlingBlockLen;
     }
 
     /**
      * @return Fragmentizer throttling delay.
      */
-    public long fragmentizerThrottlingDelay() {
+    public long getFragmentizerThrottlingDelay() {
         return fragmentizerThrottlingDelay;
     }
 
     /**
      * @return IPC endpoint config to publish IGFS over.
      */
-    @Nullable public String ipcEndpointConfiguration() {
+    @Nullable public String getIpcEndpointConfiguration() {
         return ipcEndpointCfg;
     }
 
     /**
      * @return IPC endpoint enabled flag.
      */
-    public boolean ipcEndpointEnabled() {
+    public boolean isIpcEndpointEnabled() {
         return ipcEndpointEnabled;
     }
 
     /**
      * @return Maximum space.
      */
-    public long maxSpace() {
+    public long getMaxSpace() {
         return maxSpace;
     }
 
     /**
      * @return Management port.
      */
-    public int managementPort() {
+    public int getManagementPort() {
         return mgmtPort;
     }
 
     /**
      * @return Amount of sequential block reads before prefetch is triggered.
      */
-    public int sequenceReadsBeforePrefetch() {
+    public int getSequenceReadsBeforePrefetch() {
         return seqReadsBeforePrefetch;
     }
 
     /**
      * @return Trash purge await timeout.
      */
-    public long trashPurgeTimeout() {
+    public long getTrashPurgeTimeout() {
         return trashPurgeTimeout;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, name);
+        U.writeString(out, metaCacheName);
+        U.writeString(out, dataCacheName);
+        out.writeInt(blockSize);
+        out.writeInt(prefetchBlocks);
+        out.writeInt(streamBufSize);
+        out.writeInt(perNodeBatchSize);
+        out.writeInt(perNodeParallelBatchCnt);
+        U.writeEnum(out, dfltMode);
+        U.writeMap(out, pathModes);
+        U.writeString(out, dualModePutExecutorSrvc);
+        out.writeBoolean(dualModePutExecutorSrvcShutdown);
+        out.writeLong(dualModeMaxPendingPutsSize);
+        out.writeLong(maxTaskRangeLen);
+        out.writeInt(fragmentizerConcurrentFiles);
+        out.writeFloat(fragmentizerLocWritesRatio);
+        out.writeBoolean(fragmentizerEnabled);
+        out.writeLong(fragmentizerThrottlingBlockLen);
+        out.writeLong(fragmentizerThrottlingDelay);
+        U.writeString(out, ipcEndpointCfg);
+        out.writeBoolean(ipcEndpointEnabled);
+        out.writeLong(maxSpace);
+        out.writeInt(mgmtPort);
+        out.writeInt(seqReadsBeforePrefetch);
+        out.writeLong(trashPurgeTimeout);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        name = U.readString(in);
+        metaCacheName = U.readString(in);
+        dataCacheName = U.readString(in);
+        blockSize = in.readInt();
+        prefetchBlocks = in.readInt();
+        streamBufSize = in.readInt();
+        perNodeBatchSize = in.readInt();
+        perNodeParallelBatchCnt = in.readInt();
+        dfltMode = IgfsMode.fromOrdinal(in.readByte());
+        pathModes = U.readMap(in);
+        dualModePutExecutorSrvc = U.readString(in);
+        dualModePutExecutorSrvcShutdown = in.readBoolean();
+        dualModeMaxPendingPutsSize = in.readLong();
+        maxTaskRangeLen = in.readLong();
+        fragmentizerConcurrentFiles = in.readInt();
+        fragmentizerLocWritesRatio = in.readFloat();
+        fragmentizerEnabled = in.readBoolean();
+        fragmentizerThrottlingBlockLen = in.readLong();
+        fragmentizerThrottlingDelay = in.readLong();
+        ipcEndpointCfg = U.readString(in);
+        ipcEndpointEnabled = in.readBoolean();
+        maxSpace = in.readLong();
+        mgmtPort = in.readInt();
+        seqReadsBeforePrefetch = in.readInt();
+        trashPurgeTimeout = in.readLong();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorIgfsConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorLifecycleConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorLifecycleConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorLifecycleConfiguration.java
index 838a6cc..5407699 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorLifecycleConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorLifecycleConfiguration.java
@@ -17,10 +17,13 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.configuration.IgniteConfiguration;
-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.compactArray;
@@ -28,7 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.compactArray;
 /**
  * Data transfer object for node lifecycle configuration properties.
  */
-public class VisorLifecycleConfiguration implements Serializable, LessNamingBean {
+public class VisorLifecycleConfiguration extends VisorDataTransferObject{
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -36,25 +39,39 @@ public class VisorLifecycleConfiguration implements Serializable, LessNamingBean
     private String beans;
 
     /**
-     * @param c Grid configuration.
-     * @return Data transfer object for node lifecycle configuration properties.
+     * Default constructor.
      */
-    public static VisorLifecycleConfiguration from(IgniteConfiguration c) {
-        VisorLifecycleConfiguration cfg = new VisorLifecycleConfiguration();
-
-        cfg.beans = compactArray(c.getLifecycleBeans());
+    public VisorLifecycleConfiguration() {
+        // No-op.
+    }
 
-        return cfg;
+    /**
+     * Create data transfer object for node lifecycle configuration properties.
+     *
+     * @param c Grid configuration.
+     */
+    public VisorLifecycleConfiguration(IgniteConfiguration c) {
+        beans = compactArray(c.getLifecycleBeans());
     }
 
     /**
      * @return Lifecycle beans.
      */
-    @Nullable public String beans() {
+    @Nullable public String getBeans() {
         return beans;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, beans);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        beans = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorLifecycleConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
index f380394..7a0bc76 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryConfiguration.java
@@ -17,25 +17,27 @@
 
 package org.apache.ignite.internal.visor.node;
 
-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.configuration.MemoryConfiguration;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
 import org.apache.ignite.internal.util.typedef.F;
+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 memory configuration.
  */
-public class VisorMemoryConfiguration implements Serializable {
+public class VisorMemoryConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** File cache allocation path. */
-    private String fileCacheAllocationPath;
-
-    /** Amount of memory allocated for the page cache. */
-    private long pageCacheSize;
+    /** Size of memory for system cache. */
+    private long sysCacheMemSize;
 
     /** Page size. */
     private int pageSize;
@@ -50,6 +52,13 @@ public class VisorMemoryConfiguration implements Serializable {
     private List<VisorMemoryPolicyConfiguration> memPlcs;
 
     /**
+     * Default constructor.
+     */
+    public VisorMemoryConfiguration() {
+        // No-op.
+    }
+
+    /**
      * Create data transfer object.
      *
      * @param memCfg Memory configuration.
@@ -57,6 +66,7 @@ public class VisorMemoryConfiguration implements Serializable {
     public VisorMemoryConfiguration(MemoryConfiguration memCfg) {
         assert memCfg != null;
 
+        sysCacheMemSize = memCfg.getSystemCacheMemorySize();
         pageSize = memCfg.getPageSize();
         concLvl = memCfg.getConcurrencyLevel();
         dfltMemPlcName = memCfg.getDefaultMemoryPolicyName();
@@ -79,17 +89,10 @@ public class VisorMemoryConfiguration implements Serializable {
     }
 
     /**
-     * @return File allocation path.
+     * @return Size of memory for system cache.
      */
-    public String fileCacheAllocationPath() {
-        return fileCacheAllocationPath;
-    }
-
-    /**
-     * @return Page cache size, in bytes.
-     */
-    public long pageCacheSize() {
-        return pageCacheSize;
+    public long getSystemCacheMemorySize() {
+        return sysCacheMemSize;
     }
 
     /**
@@ -112,4 +115,27 @@ public class VisorMemoryConfiguration implements Serializable {
     public List<VisorMemoryPolicyConfiguration> getMemoryPolicies() {
         return memPlcs;
     }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeLong(sysCacheMemSize);
+        out.writeInt(pageSize);
+        out.writeInt(concLvl);
+        U.writeString(out, dfltMemPlcName);
+        U.writeCollection(out, memPlcs);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        sysCacheMemSize = in.readLong();
+        pageSize = in.readInt();
+        concLvl = in.readInt();
+        dfltMemPlcName = U.readString(in);
+        memPlcs = U.readList(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorMemoryConfiguration.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
index 28eebd2..509aa48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMemoryPolicyConfiguration.java
@@ -17,13 +17,18 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.configuration.MemoryPolicyConfiguration;
+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 memory configuration.
  */
-public class VisorMemoryPolicyConfiguration implements Serializable {
+public class VisorMemoryPolicyConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -36,6 +41,14 @@ public class VisorMemoryPolicyConfiguration implements Serializable {
     /** Path for memory mapped file. */
     private String swapFilePath;
 
+
+    /**
+     * Default constructor.
+     */
+    public VisorMemoryPolicyConfiguration() {
+        // No-op.
+    }
+
     /**
      * Constructor.
      *
@@ -69,4 +82,24 @@ public class VisorMemoryPolicyConfiguration implements Serializable {
     public String getSwapFilePath() {
         return swapFilePath;
     }
+
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, name);
+        out.writeLong(size);
+        U.writeString(out, swapFilePath);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        name = U.readString(in);
+        size = in.readLong();
+        swapFilePath = U.readString(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorMemoryPolicyConfiguration.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMetricsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMetricsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMetricsConfiguration.java
index 4e812b7..4ce7b6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMetricsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorMetricsConfiguration.java
@@ -17,15 +17,17 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data transfer object for node metrics configuration properties.
  */
-public class VisorMetricsConfiguration implements Serializable, LessNamingBean {
+public class VisorMetricsConfiguration extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -39,41 +41,59 @@ public class VisorMetricsConfiguration implements Serializable, LessNamingBean {
     private long logFreq;
 
     /**
-     * @param c Grid configuration.
-     * @return Data transfer object for node metrics configuration properties.
+     * Default constructor.
      */
-    public static VisorMetricsConfiguration from(IgniteConfiguration c) {
-        VisorMetricsConfiguration cfg = new VisorMetricsConfiguration();
-
-        cfg.expTime = c.getMetricsExpireTime();
-        cfg.histSize = c.getMetricsHistorySize();
-        cfg.logFreq = c.getMetricsLogFrequency();
+    public VisorMetricsConfiguration() {
+        // No-op.
+    }
 
-        return cfg;
+    /**
+     * Create transfer object for node metrics configuration properties.
+     *
+     * @param c Grid configuration.
+     */
+    public VisorMetricsConfiguration(IgniteConfiguration c) {
+        expTime = c.getMetricsExpireTime();
+        histSize = c.getMetricsHistorySize();
+        logFreq = c.getMetricsLogFrequency();
     }
 
     /**
      * @return Metrics expired time.
      */
-    public long expireTime() {
+    public long getExpireTime() {
         return expTime;
     }
 
     /**
      * @return Number of node metrics stored in memory.
      */
-    public int historySize() {
+    public int getHistorySize() {
         return histSize;
     }
 
     /**
      * @return Frequency of metrics log printout.
      */
-    public long loggerFrequency() {
+    public long getLoggerFrequency() {
         return logFreq;
     }
 
     /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeLong(expTime);
+        out.writeInt(histSize);
+        out.writeLong(logFreq);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        expTime = in.readLong();
+        histSize = in.readInt();
+        logFreq = in.readLong();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorMetricsConfiguration.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeConfigurationCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeConfigurationCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeConfigurationCollectorJob.java
index 109b1d8..35b7ad3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeConfigurationCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeConfigurationCollectorJob.java
@@ -37,11 +37,11 @@ public class VisorNodeConfigurationCollectorJob extends VisorJob<Void, VisorGrid
 
     /** {@inheritDoc} */
     @Override protected VisorGridConfiguration run(Void arg) {
-        return new VisorGridConfiguration().from(ignite);
+        return new VisorGridConfiguration(ignite);
     }
 
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorNodeConfigurationCollectorJob.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/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 0617e70..125e219 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
@@ -24,6 +24,8 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorAdapter;
 import org.apache.ignite.internal.util.ipc.IpcServerEndpoint;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -33,6 +35,7 @@ import org.apache.ignite.internal.visor.cache.VisorCache;
 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;
@@ -70,7 +73,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, EVT_MAPPER));
+        res.getEvents().addAll(collectEvents(ignite, evtOrderKey, evtThrottleCntrKey, all, EVT_MAPPER));
     }
 
     /**
@@ -83,12 +86,12 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         try {
             // Visor events explicitly enabled in configuration.
             if (checkExplicitTaskMonitoring(ignite))
-                res.taskMonitoringEnabled(true);
+                res.setTaskMonitoringEnabled(true);
             else {
                 // Get current task monitoring state.
-                res.taskMonitoringEnabled(arg.taskMonitoringEnabled());
+                res.setTaskMonitoringEnabled(arg.isTaskMonitoringEnabled());
 
-                if (arg.taskMonitoringEnabled()) {
+                if (arg.isTaskMonitoringEnabled()) {
                     ConcurrentMap<String, VisorComputeMonitoringHolder> storage = ignite.cluster().nodeLocalMap();
 
                     VisorComputeMonitoringHolder holder = storage.get(COMPUTE_MONITORING_HOLDER_KEY);
@@ -102,21 +105,33 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                     }
 
                     // Enable task monitoring for new node in grid.
-                    holder.startCollect(ignite, arg.eventsOrderKey());
+                    holder.startCollect(ignite, arg.getEventsOrderKey());
 
                     // Update current state after change (it may not changed in some cases).
-                    res.taskMonitoringEnabled(ignite.allEventsUserRecordable(VISOR_TASK_EVTS));
+                    res.setTaskMonitoringEnabled(ignite.allEventsUserRecordable(VISOR_TASK_EVTS));
                 }
             }
 
-            events0(res, arg.eventsOrderKey(), arg.eventsThrottleCounterKey(), arg.taskMonitoringEnabled());
+            events0(res, arg.getEventsOrderKey(), arg.getEventsThrottleCounterKey(), arg.isTaskMonitoringEnabled());
         }
         catch (Exception eventsEx) {
-            res.eventsEx(eventsEx);
+            res.setEventsEx(eventsEx);
         }
     }
 
     /**
+     * @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;
+    }
+
+    /**
      * @param cacheName Cache name to check.
      * @return {@code true} if cache on local node is not a data cache or near cache disabled.
      */
@@ -138,19 +153,22 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         try {
             IgniteConfiguration cfg = ignite.configuration();
 
-            for (String cacheName : ignite.context().cache().cacheNames()) {
+            GridCacheProcessor cacheProc = ignite.context().cache();
+
+            for (String cacheName : cacheProc.cacheNames()) {
                 if (proxyCache(cacheName))
                     continue;
 
-                if (arg.systemCaches() ||
-                    !(isSystemCache(cacheName) || isIgfsCache(cfg, cacheName))) {
+                if (arg.getSystemCaches() || !(isSystemCache(cacheName) || isIgfsCache(cfg, cacheName))) {
                     long start0 = U.currentTimeMillis();
 
                     try {
-                        VisorCache cache = new VisorCache().from(ignite, cacheName, arg.sample());
+                        GridCacheAdapter ca = cacheProc.internalCache(cacheName);
+
+                        if (ca == null || !ca.context().started())
+                            continue;
 
-                        if (cache != null)
-                            res.caches().add(cache);
+                        res.getCaches().add(new VisorCache(ignite, ca, arg.getSample()));
                     }
                     catch(IllegalStateException | IllegalArgumentException e) {
                         if (debug && ignite.log() != null)
@@ -164,7 +182,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception cachesEx) {
-            res.cachesEx(cachesEx);
+            res.setCachesEx(cachesEx);
         }
     }
 
@@ -182,8 +200,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
 
                 FileSystemConfiguration igfsCfg = igfs.configuration();
 
-                if (proxyCache(igfsCfg.getDataCacheConfiguration().getName())
-                    || proxyCache(igfsCfg.getMetaCacheConfiguration().getName()))
+                if (proxyCache(igfsCfg.getDataCacheConfiguration().getName()) || proxyCache(igfsCfg.getMetaCacheConfiguration().getName()))
                     continue;
 
                 try {
@@ -192,11 +209,11 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
                     if (endPoints != null) {
                         for (IpcServerEndpoint ep : endPoints)
                             if (ep.isManagement())
-                                res.igfsEndpoints().add(new VisorIgfsEndpoint(igfs.name(), ignite.name(),
+                                res.getIgfsEndpoints().add(new VisorIgfsEndpoint(igfs.name(), ignite.name(),
                                     ep.getHost(), ep.getPort()));
                     }
 
-                    res.igfss().add(VisorIgfs.from(igfs));
+                    res.getIgfss().add(new VisorIgfs(igfs));
                 }
                 finally {
                     if (debug)
@@ -205,7 +222,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
             }
         }
         catch (Exception igfssEx) {
-            res.igfssEx(igfssEx);
+            res.setIgfssEx(igfssEx);
         }
     }
 
@@ -223,9 +240,9 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
      */
     protected VisorNodeDataCollectorJobResult run(VisorNodeDataCollectorJobResult res,
         VisorNodeDataCollectorTaskArg arg) {
-        res.igniteInstanceName(ignite.name());
+        res.setGridName(ignite.name());
 
-        res.topologyVersion(ignite.cluster().topologyVersion());
+        res.setTopologyVersion(ignite.cluster().topologyVersion());
 
         long start0 = U.currentTimeMillis();
 
@@ -244,7 +261,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         if (debug)
             log(ignite.log(), "Collected igfs", getClass(), start0);
 
-        res.errorCount(ignite.context().exceptionRegistry().errorCount());
+        res.setErrorCount(ignite.context().exceptionRegistry().errorCount());
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
index d463baa..41f9468 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJobResult.java
@@ -17,9 +17,14 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import java.util.ArrayList;
-import java.util.Collection;
+import java.util.List;
+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.apache.ignite.internal.visor.cache.VisorCache;
 import org.apache.ignite.internal.visor.event.VisorGridEvent;
 import org.apache.ignite.internal.visor.igfs.VisorIgfs;
@@ -28,12 +33,12 @@ import org.apache.ignite.internal.visor.igfs.VisorIgfsEndpoint;
 /**
  * Data collector job result.
  */
-public class VisorNodeDataCollectorJobResult implements Serializable {
+public class VisorNodeDataCollectorJobResult extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Ignite instance name. */
-    private String igniteInstanceName;
+    /** Grid name. */
+    private String gridName;
 
     /** Node topology version. */
     private long topVer;
@@ -42,22 +47,22 @@ public class VisorNodeDataCollectorJobResult implements Serializable {
     private boolean taskMonitoringEnabled;
 
     /** Node events. */
-    private final Collection<VisorGridEvent> evts = new ArrayList<>();
+    private List<VisorGridEvent> evts = new ArrayList<>();
 
     /** Exception while collecting node events. */
     private Throwable evtsEx;
 
     /** Node caches. */
-    private final Collection<VisorCache> caches = new ArrayList<>();
+    private List<VisorCache> caches = new ArrayList<>();
 
     /** Exception while collecting node caches. */
     private Throwable cachesEx;
 
     /** Node IGFSs. */
-    private final Collection<VisorIgfs> igfss = new ArrayList<>();
+    private List<VisorIgfs> igfss = new ArrayList<>();
 
     /** All IGFS endpoints collected from nodes. */
-    private final Collection<VisorIgfsEndpoint> igfsEndpoints = new ArrayList<>();
+    private List<VisorIgfsEndpoint> igfsEndpoints = new ArrayList<>();
 
     /** Exception while collecting node IGFSs. */
     private Throwable igfssEx;
@@ -66,128 +71,170 @@ public class VisorNodeDataCollectorJobResult implements Serializable {
     private long errCnt;
 
     /**
-     * @return Ignite instance name.
+     * Default constructor.
      */
-    public String igniteInstanceName() {
-        return igniteInstanceName;
+    public VisorNodeDataCollectorJobResult() {
+        // No-op.
     }
 
     /**
-     * @param igniteInstanceName New Ignite instance name.
+     * @return Grid name.
      */
-    public void igniteInstanceName(String igniteInstanceName) {
-        this.igniteInstanceName = igniteInstanceName;
+    public String getGridName() {
+        return gridName;
+    }
+
+    /**
+     * @param gridName New grid name value.
+     */
+    public void setGridName(String gridName) {
+        this.gridName = gridName;
     }
 
     /**
      * @return Current topology version.
      */
-    public long topologyVersion() {
+    public long getTopologyVersion() {
         return topVer;
     }
 
     /**
      * @param topVer New topology version value.
      */
-    public void topologyVersion(long topVer) {
+    public void setTopologyVersion(long topVer) {
         this.topVer = topVer;
     }
 
     /**
      * @return Current task monitoring state.
      */
-    public boolean taskMonitoringEnabled() {
+    public boolean isTaskMonitoringEnabled() {
         return taskMonitoringEnabled;
     }
 
     /**
      * @param taskMonitoringEnabled New value of task monitoring state.
      */
-    public void taskMonitoringEnabled(boolean taskMonitoringEnabled) {
+    public void setTaskMonitoringEnabled(boolean taskMonitoringEnabled) {
         this.taskMonitoringEnabled = taskMonitoringEnabled;
     }
 
     /**
      * @return Collection of collected events.
      */
-    public Collection<VisorGridEvent> events() {
+    public List<VisorGridEvent> getEvents() {
         return evts;
     }
 
     /**
      * @return Exception caught during collecting events.
      */
-    public Throwable eventsEx() {
+    public Throwable getEventsEx() {
         return evtsEx;
     }
 
     /**
      * @param evtsEx Exception caught during collecting events.
      */
-    public void eventsEx(Throwable evtsEx) {
+    public void setEventsEx(Throwable evtsEx) {
         this.evtsEx = evtsEx;
     }
 
     /**
      * @return Collected cache metrics.
      */
-    public Collection<VisorCache> caches() {
+    public List<VisorCache> getCaches() {
         return caches;
     }
 
     /**
      * @return Exception caught during collecting caches metrics.
      */
-    public Throwable cachesEx() {
+    public Throwable getCachesEx() {
         return cachesEx;
     }
 
     /**
      * @param cachesEx Exception caught during collecting caches metrics.
      */
-    public void cachesEx(Throwable cachesEx) {
+    public void setCachesEx(Throwable cachesEx) {
         this.cachesEx = cachesEx;
     }
 
     /**
      * @return Collected IGFSs metrics.
      */
-    public Collection<VisorIgfs> igfss() {
+    public List<VisorIgfs> getIgfss() {
         return igfss;
     }
 
     /**
      * @return Collected IGFSs endpoints.
      */
-    public Collection<VisorIgfsEndpoint> igfsEndpoints() {
+    public List<VisorIgfsEndpoint> getIgfsEndpoints() {
         return igfsEndpoints;
     }
 
     /**
      * @return Exception caught during collecting IGFSs metrics.
      */
-    public Throwable igfssEx() {
+    public Throwable getIgfssEx() {
         return igfssEx;
     }
 
     /**
      * @param igfssEx Exception caught during collecting IGFSs metrics.
      */
-    public void igfssEx(Throwable igfssEx) {
+    public void setIgfssEx(Throwable igfssEx) {
         this.igfssEx = igfssEx;
     }
 
     /**
      * @return Errors count.
      */
-    public long errorCount() {
+    public long getErrorCount() {
         return errCnt;
     }
 
     /**
      * @param errCnt Errors count.
      */
-    public void errorCount(long errCnt) {
+    public void setErrorCount(long errCnt) {
         this.errCnt = errCnt;
     }
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        U.writeString(out, gridName);
+        out.writeLong(topVer);
+        out.writeBoolean(taskMonitoringEnabled);
+        U.writeCollection(out, evts);
+        out.writeObject(evtsEx);
+        U.writeCollection(out, caches);
+        out.writeObject(cachesEx);
+        U.writeCollection(out, igfss);
+        U.writeCollection(out, igfsEndpoints);
+        out.writeObject(igfssEx);
+        out.writeLong(errCnt);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        gridName = U.readString(in);
+        topVer = in.readLong();
+        taskMonitoringEnabled = in.readBoolean();
+        evts = U.readList(in);
+        evtsEx = (Throwable)in.readObject();
+        caches = U.readList(in);
+        cachesEx = (Throwable)in.readObject();
+        igfss = U.readList(in);
+        igfsEndpoints = U.readList(in);
+        igfssEx = (Throwable)in.readObject();
+        errCnt = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorNodeDataCollectorJobResult.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index 08d5f9f..af02e34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -66,11 +66,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
                 else {
                     // Ignore nodes that left topology.
                     if (!(unhandledEx instanceof ClusterGroupEmptyException))
-                        taskRes.unhandledEx().put(nid, new VisorExceptionWrapper(unhandledEx));
+                        taskRes.getUnhandledEx().put(nid, new VisorExceptionWrapper(unhandledEx));
                 }
             }
         }
 
+        taskRes.active(true /*ignite.active()*/);
+
         taskRes.active(ignite.active());
 
         return taskRes;
@@ -85,33 +87,33 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
      */
     protected void reduceJobResult(VisorNodeDataCollectorTaskResult taskRes,
         VisorNodeDataCollectorJobResult jobRes, UUID nid) {
-        taskRes.igniteInstanceNames().put(nid, jobRes.igniteInstanceName());
+        taskRes.getGridNames().put(nid, jobRes.getGridName());
 
-        taskRes.topologyVersions().put(nid, jobRes.topologyVersion());
+        taskRes.getTopologyVersions().put(nid, jobRes.getTopologyVersion());
 
-        taskRes.taskMonitoringEnabled().put(nid, jobRes.taskMonitoringEnabled());
+        taskRes.isTaskMonitoringEnabled().put(nid, jobRes.isTaskMonitoringEnabled());
 
-        taskRes.errorCounts().put(nid, jobRes.errorCount());
+        taskRes.getErrorCounts().put(nid, jobRes.getErrorCount());
 
-        if (!jobRes.events().isEmpty())
-            taskRes.events().addAll(jobRes.events());
+        if (!jobRes.getEvents().isEmpty())
+            taskRes.getEvents().addAll(jobRes.getEvents());
 
-        if (jobRes.eventsEx() != null)
-            taskRes.eventsEx().put(nid, new VisorExceptionWrapper(jobRes.eventsEx()));
+        if (jobRes.getEventsEx() != null)
+            taskRes.getEventsEx().put(nid, new VisorExceptionWrapper(jobRes.getEventsEx()));
 
-        if (!jobRes.caches().isEmpty())
-            taskRes.caches().put(nid, jobRes.caches());
+        if (!jobRes.getCaches().isEmpty())
+            taskRes.getCaches().put(nid, jobRes.getCaches());
 
-        if (jobRes.cachesEx() != null)
-            taskRes.cachesEx().put(nid, new VisorExceptionWrapper(jobRes.cachesEx()));
+        if (jobRes.getCachesEx() != null)
+            taskRes.getCachesEx().put(nid, new VisorExceptionWrapper(jobRes.getCachesEx()));
 
-        if (!jobRes.igfss().isEmpty())
-            taskRes.igfss().put(nid, jobRes.igfss());
+        if (!jobRes.getIgfss().isEmpty())
+            taskRes.getIgfss().put(nid, jobRes.getIgfss());
 
-        if (!jobRes.igfsEndpoints().isEmpty())
-            taskRes.igfsEndpoints().put(nid, jobRes.igfsEndpoints());
+        if (!jobRes.getIgfsEndpoints().isEmpty())
+            taskRes.getIgfsEndpoints().put(nid, jobRes.getIgfsEndpoints());
 
-        if (jobRes.igfssEx() != null)
-            taskRes.igfssEx().put(nid, new VisorExceptionWrapper(jobRes.igfssEx()));
+        if (jobRes.getIgfssEx() != null)
+            taskRes.getIgfssEx().put(nid, new VisorExceptionWrapper(jobRes.getIgfssEx()));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/12dfe9e8/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskArg.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskArg.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskArg.java
index 975dd28..825560b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskArg.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskArg.java
@@ -17,12 +17,17 @@
 
 package org.apache.ignite.internal.visor.node;
 
-import java.io.Serializable;
+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.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorDataTransferObject;
 
 /**
  * Data collector task arguments.
  */
-public class VisorNodeDataCollectorTaskArg implements Serializable {
+public class VisorNodeDataCollectorTaskArg extends VisorDataTransferObject {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -74,70 +79,93 @@ public class VisorNodeDataCollectorTaskArg implements Serializable {
     /**
      * @return {@code true} if Visor should collect information about tasks.
      */
-    public boolean taskMonitoringEnabled() {
+    public boolean isTaskMonitoringEnabled() {
         return taskMonitoringEnabled;
     }
 
     /**
      * @param taskMonitoringEnabled If {@code true} then Visor should collect information about tasks.
      */
-    public void taskMonitoringEnabled(boolean taskMonitoringEnabled) {
+    public void setTaskMonitoringEnabled(boolean taskMonitoringEnabled) {
         this.taskMonitoringEnabled = taskMonitoringEnabled;
     }
 
     /**
      * @return Key for store and read last event order number.
      */
-    public String eventsOrderKey() {
+    public String getEventsOrderKey() {
         return evtOrderKey;
     }
 
     /**
      * @param evtOrderKey Key for store and read last event order number.
      */
-    public void eventsOrderKey(String evtOrderKey) {
+    public void setEventsOrderKey(String evtOrderKey) {
         this.evtOrderKey = evtOrderKey;
     }
 
     /**
      * @return Key for store and read events throttle counter.
      */
-    public String eventsThrottleCounterKey() {
+    public String getEventsThrottleCounterKey() {
         return evtThrottleCntrKey;
     }
 
     /**
      * @param evtThrottleCntrKey Key for store and read events throttle counter.
      */
-    public void eventsThrottleCounterKey(String evtThrottleCntrKey) {
+    public void setEventsThrottleCounterKey(String evtThrottleCntrKey) {
         this.evtThrottleCntrKey = evtThrottleCntrKey;
     }
 
     /**
      * @return Number of items to evaluate cache size.
      */
-    public int sample() {
+    public int getSample() {
         return sample;
     }
 
     /**
      * @param sample Number of items to evaluate cache size.
      */
-    public void sample(int sample) {
+    public void setSample(int sample) {
         this.sample = sample;
     }
 
     /**
      * @return {@code true} if Visor should collect metrics for system caches.
      */
-    public boolean systemCaches() {
+    public boolean getSystemCaches() {
         return sysCaches;
     }
 
     /**
      * @param sysCaches {@code true} if Visor should collect metrics for system caches.
      */
-    public void systemCaches(boolean sysCaches) {
+    public void setSystemCaches(boolean sysCaches) {
         this.sysCaches = sysCaches;
     }
-}
\ No newline at end of file
+
+    /** {@inheritDoc} */
+    @Override protected void writeExternalData(ObjectOutput out) throws IOException {
+        out.writeBoolean(taskMonitoringEnabled);
+        U.writeString(out, evtOrderKey);
+        U.writeString(out, evtThrottleCntrKey);
+        out.writeInt(sample);
+        out.writeBoolean(sysCaches);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readExternalData(byte protoVer, ObjectInput in) throws IOException, ClassNotFoundException {
+        taskMonitoringEnabled = in.readBoolean();
+        evtOrderKey = U.readString(in);
+        evtThrottleCntrKey = U.readString(in);
+        sample = in.readInt();
+        sysCaches = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorNodeDataCollectorTaskArg.class, this);
+    }
+}