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 2015/03/02 10:46:57 UTC

[1/8] incubator-ignite git commit: # ignite-339 Refactored VisorCacheAffinityConfiguration

Repository: incubator-ignite
Updated Branches:
  refs/heads/sprint-2 13e2d1f67 -> c847e8851


# ignite-339 Refactored VisorCacheAffinityConfiguration


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

Branch: refs/heads/sprint-2
Commit: ff0c1e1dbbd6da1a8724b4b40ecbf1095f506be9
Parents: adb2454
Author: anovikov <an...@gridgain.com>
Authored: Fri Feb 27 15:49:47 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Fri Feb 27 15:49:47 2015 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCache.java | 190 ++------
 .../cache/VisorCacheAffinityConfiguration.java  |  53 +--
 .../visor/cache/VisorCacheConfiguration.java    | 462 ++++---------------
 .../cache/VisorCacheDefaultConfiguration.java   |  27 +-
 .../cache/VisorCacheEvictionConfiguration.java  |  81 +---
 .../cache/VisorCacheNearConfiguration.java      |  42 +-
 .../cache/VisorCachePreloadConfiguration.java   |  54 +--
 .../cache/VisorCacheStoreConfiguration.java     | 148 +++++-
 .../VisorCacheWriteBehindConfiguration.java     | 137 ------
 .../internal/visor/util/VisorTaskUtils.java     |   4 +-
 .../commands/cache/VisorCacheCommand.scala      |  26 +-
 11 files changed, 304 insertions(+), 920 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index 72a93e0..500e517 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -77,19 +77,19 @@ public class VisorCache implements Serializable {
     private long swapKeys;
 
     /** Number of partitions. */
-    private int partsCnt;
+    private int partitions;
 
     /** Primary partitions IDs with sizes. */
-    private Collection<IgnitePair<Integer>> primaryParts;
+    private Collection<IgnitePair<Integer>> primaryPartitions;
 
     /** Backup partitions IDs with sizes. */
-    private Collection<IgnitePair<Integer>> backupParts;
+    private Collection<IgnitePair<Integer>> backupPartitions;
 
     /** Cache metrics. */
     private VisorCacheMetrics metrics;
 
     /** Cache partitions states. */
-    private GridDhtPartitionMap partsMap;
+    private GridDhtPartitionMap partitionsMap;
 
     /**
      * @param ignite Grid.
@@ -209,22 +209,22 @@ public class VisorCache implements Serializable {
 
         VisorCache cache = new VisorCache();
 
-        cache.name(cacheName);
-        cache.mode(mode);
-        cache.memorySize(memSz);
-        cache.size(size);
-        cache.nearSize(near);
-        cache.dhtSize(size - near);
-        cache.primarySize(ca.primarySize());
-        cache.offHeapAllocatedSize(ca.offHeapAllocatedSize());
-        cache.offHeapEntriesCount(ca.offHeapEntriesCount());
-        cache.swapSize(swapSize);
-        cache.swapKeys(swapKeys);
-        cache.partitions(ca.affinity().partitions());
-        cache.primaryPartitions(pps);
-        cache.backupPartitions(bps);
-        cache.metrics(VisorCacheMetrics.from(ca));
-        cache.partitionMap(partsMap);
+        cache.name = cacheName;
+        cache.mode = mode;
+        cache.memorySize = memSz;
+        cache.size = size;
+        cache.nearSize = near;
+        cache.dhtSize = size - near;
+        cache.primarySize = ca.primarySize();
+        cache.offHeapAllocatedSize = ca.offHeapAllocatedSize();
+        cache.offHeapEntriesCnt = ca.offHeapEntriesCount();
+        cache.swapSize = swapSize;
+        cache.swapKeys = swapKeys;
+        cache.partitions = ca.affinity().partitions();
+        cache.primaryPartitions = pps;
+        cache.backupPartitions = bps;
+        cache.metrics = VisorCacheMetrics.from(ca);
+        cache.partitionsMap = partsMap;
 
         return cache;
     }
@@ -235,21 +235,21 @@ public class VisorCache implements Serializable {
     public VisorCache history() {
         VisorCache c = new VisorCache();
 
-        c.name(name);
-        c.mode(mode);
-        c.memorySize(memorySize);
-        c.size(size);
-        c.nearSize(nearSize);
-        c.dhtSize(dhtSize);
-        c.primarySize(primarySize);
-        c.offHeapAllocatedSize(offHeapAllocatedSize);
-        c.offHeapEntriesCount(offHeapEntriesCnt);
-        c.swapSize(swapSize);
-        c.swapKeys(swapKeys);
-        c.partitions(partsCnt);
-        c.primaryPartitions(Collections.<IgnitePair<Integer>>emptyList());
-        c.backupPartitions(Collections.<IgnitePair<Integer>>emptyList());
-        c.metrics(metrics);
+        c.name = name;
+        c.mode = mode;
+        c.memorySize = memorySize;
+        c.size = size;
+        c.nearSize = nearSize;
+        c.dhtSize = dhtSize;
+        c.primarySize = primarySize;
+        c.offHeapAllocatedSize = offHeapAllocatedSize;
+        c.offHeapEntriesCnt = offHeapEntriesCnt;
+        c.swapSize = swapSize;
+        c.swapKeys = swapKeys;
+        c.partitions = partitions;
+        c.primaryPartitions = Collections.emptyList();
+        c.backupPartitions = Collections.emptyList();
+        c.metrics = metrics;
 
         return c;
     }
@@ -262,13 +262,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param name New cache name.
-     */
-    public void name(String name) {
-        this.name = name;
-    }
-
-    /**
      * @return Cache mode.
      */
     public CacheMode mode() {
@@ -276,13 +269,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param mode New cache mode.
-     */
-    public void mode(CacheMode mode) {
-        this.mode = mode;
-    }
-
-    /**
      * @return Cache size in bytes.
      */
     public long memorySize() {
@@ -290,13 +276,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param memorySize New cache size in bytes.
-     */
-    public void memorySize(long memorySize) {
-        this.memorySize = memorySize;
-    }
-
-    /**
      * @return Number of all entries in cache.
      */
     public int size() {
@@ -304,13 +283,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param size New number of all entries in cache.
-     */
-    public void size(int size) {
-        this.size = size;
-    }
-
-    /**
      * @return Number of all entries in near cache.
      */
     public int nearSize() {
@@ -318,13 +290,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param nearSize New number of all entries in near cache.
-     */
-    public void nearSize(int nearSize) {
-        this.nearSize = nearSize;
-    }
-
-    /**
      * @return Number of all entries in DHT cache.
      */
     public int dhtSize() {
@@ -332,13 +297,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param dhtSize New number of all entries in DHT cache.
-     */
-    public void dhtSize(int dhtSize) {
-        this.dhtSize = dhtSize;
-    }
-
-    /**
      * @return Number of primary entries in cache.
      */
     public int primarySize() {
@@ -346,13 +304,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param primarySize New number of primary entries in cache.
-     */
-    public void primarySize(int primarySize) {
-        this.primarySize = primarySize;
-    }
-
-    /**
      * @return Memory size allocated in off-heap.
      */
     public long offHeapAllocatedSize() {
@@ -360,13 +311,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param offHeapAllocatedSize New memory size allocated in off-heap.
-     */
-    public void offHeapAllocatedSize(long offHeapAllocatedSize) {
-        this.offHeapAllocatedSize = offHeapAllocatedSize;
-    }
-
-    /**
      * @return Number of cache entries stored in off-heap memory.
      */
     public long offHeapEntriesCount() {
@@ -374,13 +318,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param offHeapEntriesCnt New number of cache entries stored in off-heap memory.
-     */
-    public void offHeapEntriesCount(long offHeapEntriesCnt) {
-        this.offHeapEntriesCnt = offHeapEntriesCnt;
-    }
-
-    /**
      * @return Size in bytes for swap space.
      */
     public long swapSize() {
@@ -388,13 +325,6 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param swapSize New size in bytes for swap space.
-     */
-    public void swapSize(long swapSize) {
-        this.swapSize = swapSize;
-    }
-
-    /**
      * @return Number of cache entries stored in swap space.
      */
     public long swapKeys() {
@@ -402,52 +332,24 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param swapKeys New number of cache entries stored in swap space.
-     */
-    public void swapKeys(long swapKeys) {
-        this.swapKeys = swapKeys;
-    }
-
-    /**
      * @return Number of partitions.
      */
     public int partitions() {
-        return partsCnt;
-    }
-
-    /**
-     * @param partsCnt New number of partitions.
-     */
-    public void partitions(int partsCnt) {
-        this.partsCnt = partsCnt;
+        return partitions;
     }
 
     /**
      * @return Primary partitions IDs with sizes.
      */
     public Collection<IgnitePair<Integer>> primaryPartitions() {
-        return primaryParts;
-    }
-
-    /**
-     * @param primaryParts New primary partitions IDs with sizes.
-     */
-    public void primaryPartitions(Collection<IgnitePair<Integer>> primaryParts) {
-        this.primaryParts = primaryParts;
+        return primaryPartitions;
     }
 
     /**
      * @return Backup partitions IDs with sizes.
      */
     public Collection<IgnitePair<Integer>> backupPartitions() {
-        return backupParts;
-    }
-
-    /**
-     * @param backupParts New backup partitions IDs with sizes.
-     */
-    public void backupPartitions(Collection<IgnitePair<Integer>> backupParts) {
-        this.backupParts = backupParts;
+        return backupPartitions;
     }
 
     /**
@@ -458,24 +360,10 @@ public class VisorCache implements Serializable {
     }
 
     /**
-     * @param metrics New cache metrics.
-     */
-    public void metrics(VisorCacheMetrics metrics) {
-        this.metrics = metrics;
-    }
-
-    /**
      * @return Cache partitions states.
      */
     @Nullable public GridDhtPartitionMap partitionMap() {
-        return partsMap;
-    }
-
-    /**
-     * @param partsMap New cache partitions states.
-     */
-    public void partitionMap(@Nullable GridDhtPartitionMap partsMap) {
-        this.partsMap = partsMap;
+        return partitionsMap;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
index 3d4d84a..0f6a84f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheAffinityConfiguration.java
@@ -71,11 +71,12 @@ public class VisorCacheAffinityConfiguration implements Serializable {
 
         VisorCacheAffinityConfiguration cfg = new VisorCacheAffinityConfiguration();
 
-        cfg.function(compactClass(aff));
-        cfg.mapper(compactClass(ccfg.getAffinityMapper()));
-        cfg.partitionedBackups(ccfg.getBackups());
-        cfg.defaultReplicas(dfltReplicas);
-        cfg.excludeNeighbors(excludeNeighbors);
+        cfg.function = compactClass(aff);
+        cfg.mapper = compactClass(ccfg.getAffinityMapper());
+        cfg.partitions = aff.partitions();
+        cfg.partitionedBackups = ccfg.getBackups();
+        cfg.dfltReplicas = dfltReplicas;
+        cfg.excludeNeighbors = excludeNeighbors;
 
         return cfg;
     }
@@ -88,13 +89,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @param function New cache affinity function.
-     */
-    public void function(String function) {
-        this.function = function;
-    }
-
-    /**
      * @return Cache affinity mapper.
      */
     public String mapper() {
@@ -102,13 +96,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @param mapper New cache affinity mapper.
-     */
-    public void mapper(String mapper) {
-        this.mapper = mapper;
-    }
-
-    /**
      * @return Count of key backups.
      */
     public int partitionedBackups() {
@@ -116,13 +103,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @param partitionedBackups New count of key backups.
-     */
-    public void partitionedBackups(int partitionedBackups) {
-        this.partitionedBackups = partitionedBackups;
-    }
-
-    /**
      * @return Cache affinity partitions.
      */
     public Integer partitions() {
@@ -130,13 +110,6 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @param partitions New cache affinity partitions.
-     */
-    public void partitions(Integer partitions) {
-        this.partitions = partitions;
-    }
-
-    /**
      * @return Cache partitioned affinity default replicas.
      */
     @Nullable public Integer defaultReplicas() {
@@ -144,26 +117,12 @@ public class VisorCacheAffinityConfiguration implements Serializable {
     }
 
     /**
-     * @param dfltReplicas New cache partitioned affinity default replicas.
-     */
-    public void defaultReplicas(Integer dfltReplicas) {
-        this.dfltReplicas = dfltReplicas;
-    }
-
-    /**
      * @return Cache partitioned affinity exclude neighbors.
      */
     @Nullable public Boolean excludeNeighbors() {
         return excludeNeighbors;
     }
 
-    /**
-     * @param excludeNeighbors New cache partitioned affinity exclude neighbors.
-     */
-    public void excludeNeighbors(Boolean excludeNeighbors) {
-        this.excludeNeighbors = excludeNeighbors;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheAffinityConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/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 7bfaf79..cf149f7 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
@@ -19,10 +19,7 @@ package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.node.*;
 import org.jetbrains.annotations.*;
@@ -99,39 +96,30 @@ public class VisorCacheConfiguration implements Serializable {
     /** Cache interceptor. */
     private String interceptor;
 
-    /** Cache affinity config. */
-    private VisorCacheAffinityConfiguration affinity;
+    /** Should value bytes be stored. */
+    private boolean valBytes;
+
+    /** Cache affinityCfg config. */
+    private VisorCacheAffinityConfiguration affinityCfg;
 
     /** Preload config. */
-    private VisorCachePreloadConfiguration preload;
+    private VisorCachePreloadConfiguration preloadCfg;
 
     /** Eviction config. */
-    private VisorCacheEvictionConfiguration evict;
+    private VisorCacheEvictionConfiguration evictCfg;
 
     /** Near cache config. */
-    private VisorCacheNearConfiguration near;
+    private VisorCacheNearConfiguration nearCfg;
 
     /** Default config */
-    private VisorCacheDefaultConfiguration dflt;
+    private VisorCacheDefaultConfiguration dfltCfg;
 
     /** Store config */
-    private VisorCacheStoreConfiguration store;
-
-    /** Write behind config */
-    private VisorCacheWriteBehindConfiguration writeBehind;
+    private VisorCacheStoreConfiguration storeCfg;
 
     /** Collection of type metadata. */
     private Collection<VisorCacheTypeMetadata> typeMeta;
 
-    /** Whether cache has JDBC store. */
-    private boolean jdbcStore;
-
-    /** Whether cache should operate in read-through mode. */
-    private boolean readThrough;
-
-    /** Whether cache should operate in write-through mode. */
-    private boolean writeThrough;
-
     /** Whether statistics collection is enabled. */
     private boolean statisticsEnabled;
 
@@ -156,50 +144,41 @@ public class VisorCacheConfiguration implements Serializable {
      * @return Data transfer object for cache configuration properties.
      */
     public static VisorCacheConfiguration from(Ignite ignite, CacheConfiguration ccfg) {
-        GridCacheContext cctx = ((IgniteKernal)ignite).internalCache(ccfg.getName()).context();
-
-        boolean jdbcStore = cctx.store().configuredStore() instanceof CacheAbstractJdbcStore;
-
         VisorCacheConfiguration cfg = new VisorCacheConfiguration();
 
-        cfg.name(ccfg.getName());
-        cfg.mode(ccfg.getCacheMode());
-        cfg.distributionMode(ccfg.getDistributionMode());
-        cfg.atomicityMode(ccfg.getAtomicityMode());
-        cfg.atomicWriteOrderMode(ccfg.getAtomicWriteOrderMode());
-        cfg.eagerTtl(ccfg.isEagerTtl());
-        cfg.writeSynchronizationMode(ccfg.getWriteSynchronizationMode());
-        cfg.swapEnabled(ccfg.isSwapEnabled());
-        cfg.queryIndexEnabled(ccfg.isQueryIndexEnabled());
-        cfg.invalidate(ccfg.isInvalidate());
-        cfg.startSize(ccfg.getStartSize());
-        cfg.transactionManagerLookupClassName(ccfg.getTransactionManagerLookupClassName());
-        cfg.offsetHeapMaxMemory(ccfg.getOffHeapMaxMemory());
-        cfg.maxQueryIteratorCount(ccfg.getMaximumQueryIteratorCount());
-        cfg.maxConcurrentAsyncOperations(ccfg.getMaxConcurrentAsyncOperations());
-        cfg.memoryMode(ccfg.getMemoryMode());
-        cfg.indexingSpiName(ccfg.getIndexingSpiName());
-        cfg.interceptor(compactClass(ccfg.getInterceptor()));
-        cfg.affinityConfiguration(VisorCacheAffinityConfiguration.from(ccfg));
-        cfg.preloadConfiguration(VisorCachePreloadConfiguration.from(ccfg));
-        cfg.evictConfiguration(VisorCacheEvictionConfiguration.from(ccfg));
-        cfg.nearConfiguration(VisorCacheNearConfiguration.from(ccfg));
-        cfg.defaultConfiguration(VisorCacheDefaultConfiguration.from(ccfg));
-        cfg.storeConfiguration(VisorCacheStoreConfiguration.from(ccfg));
-        cfg.writeBehind(VisorCacheWriteBehindConfiguration.from(ccfg));
-
-        cfg.typeMeta(VisorCacheTypeMetadata.list(ccfg.getTypeMetadata()));
-        cfg.jdbcStore(jdbcStore);
-
-        cfg.readThrough(ccfg.isReadThrough());
-        cfg.writeThrough(ccfg.isWriteThrough());
-        cfg.statisticsEnabled(ccfg.isStatisticsEnabled());
-        cfg.managementEnabled(ccfg.isManagementEnabled());
-        cfg.loaderFactory(compactClass(ccfg.getCacheLoaderFactory()));
-        cfg.writerFactory(compactClass(ccfg.getCacheWriterFactory()));
-        cfg.expiryPolicyFactory(compactClass(ccfg.getExpiryPolicyFactory()));
-
-        cfg.queryConfiguration(VisorCacheQueryConfiguration.from(ccfg.getQueryConfiguration()));
+        cfg.name = ccfg.getName();
+        cfg.mode = ccfg.getCacheMode();
+        cfg.distributionMode = ccfg.getDistributionMode();
+        cfg.atomicityMode = ccfg.getAtomicityMode();
+        cfg.atomicWriteOrderMode = ccfg.getAtomicWriteOrderMode();
+        cfg.eagerTtl = ccfg.isEagerTtl();
+        cfg.writeSynchronizationMode = ccfg.getWriteSynchronizationMode();
+        cfg.swapEnabled = ccfg.isSwapEnabled();
+        cfg.qryIdxEnabled = ccfg.isQueryIndexEnabled();
+        cfg.invalidate = ccfg.isInvalidate();
+        cfg.startSize = ccfg.getStartSize();
+        cfg.tmLookupClsName = ccfg.getTransactionManagerLookupClassName();
+        cfg.offHeapMaxMemory = ccfg.getOffHeapMaxMemory();
+        cfg.maxQryIterCnt = ccfg.getMaximumQueryIteratorCount();
+        cfg.maxConcurrentAsyncOps = ccfg.getMaxConcurrentAsyncOperations();
+        cfg.memoryMode = ccfg.getMemoryMode();
+        cfg.indexingSpiName = ccfg.getIndexingSpiName();
+        cfg.interceptor = compactClass(ccfg.getInterceptor());
+        cfg.valBytes = ccfg.isStoreValueBytes();
+        cfg.typeMeta = VisorCacheTypeMetadata.list(ccfg.getTypeMetadata());
+        cfg.statisticsEnabled = ccfg.isStatisticsEnabled();
+        cfg.mgmtEnabled = ccfg.isManagementEnabled();
+        cfg.ldrFactory = compactClass(ccfg.getCacheLoaderFactory());
+        cfg.writerFactory = compactClass(ccfg.getCacheWriterFactory());
+        cfg.expiryPlcFactory = compactClass(ccfg.getExpiryPolicyFactory());
+
+        cfg.affinityCfg = VisorCacheAffinityConfiguration.from(ccfg);
+        cfg.preloadCfg = VisorCachePreloadConfiguration.from(ccfg);
+        cfg.evictCfg = VisorCacheEvictionConfiguration.from(ccfg);
+        cfg.nearCfg = VisorCacheNearConfiguration.from(ccfg);
+        cfg.dfltCfg = VisorCacheDefaultConfiguration.from(ccfg);
+        cfg.storeCfg = VisorCacheStoreConfiguration.from(ignite, ccfg);
+        cfg.qryCfg = VisorCacheQueryConfiguration.from(ccfg.getQueryConfiguration());
 
         return cfg;
     }
@@ -229,13 +208,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param name New cache name.
-     */
-    public void name(@Nullable String name) {
-        this.name = name;
-    }
-
-    /**
      * @return Cache mode.
      */
     public CacheMode mode() {
@@ -243,13 +215,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param mode New cache mode.
-     */
-    public void mode(CacheMode mode) {
-        this.mode = mode;
-    }
-
-    /**
      * @return Distribution mode.
      */
     public CacheDistributionMode distributionMode() {
@@ -257,13 +222,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param distributionMode New distribution mode.
-     */
-    public void distributionMode(CacheDistributionMode distributionMode) {
-        this.distributionMode = distributionMode;
-    }
-
-    /**
      * @return Cache atomicity mode
      */
     public CacheAtomicityMode atomicityMode() {
@@ -271,13 +229,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param atomicityMode New cache atomicity mode
-     */
-    public void atomicityMode(CacheAtomicityMode atomicityMode) {
-        this.atomicityMode = atomicityMode;
-    }
-
-    /**
      * @return Cache atomic sequence reserve size
      */
     public int atomicSequenceReserveSize() {
@@ -285,13 +236,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param atomicSeqReserveSize New cache atomic sequence reserve size
-     */
-    public void atomicSequenceReserveSize(int atomicSeqReserveSize) {
-        this.atomicSeqReserveSize = atomicSeqReserveSize;
-    }
-
-    /**
      * @return Cache atomicity write ordering mode.
      */
     public CacheAtomicWriteOrderMode atomicWriteOrderMode() {
@@ -299,13 +243,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param atomicWriteOrderMode New cache atomicity write ordering mode.
-     */
-    public void atomicWriteOrderMode(CacheAtomicWriteOrderMode atomicWriteOrderMode) {
-        this.atomicWriteOrderMode = atomicWriteOrderMode;
-    }
-
-    /**
      * @return Eager ttl flag
      */
     public boolean eagerTtl() {
@@ -313,13 +250,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param eagerTtl New eager ttl flag
-     */
-    public void eagerTtl(boolean eagerTtl) {
-        this.eagerTtl = eagerTtl;
-    }
-
-    /**
      * @return Write synchronization mode.
      */
     public CacheWriteSynchronizationMode writeSynchronizationMode() {
@@ -327,13 +257,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param writeSynchronizationMode New write synchronization mode.
-     */
-    public void writeSynchronizationMode(CacheWriteSynchronizationMode writeSynchronizationMode) {
-        this.writeSynchronizationMode = writeSynchronizationMode;
-    }
-
-    /**
      * @return Sequence reserve size.
      */
     public int sequenceReserveSize() {
@@ -341,13 +264,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param seqReserveSize New sequence reserve size.
-     */
-    public void sequenceReserveSize(int seqReserveSize) {
-        this.seqReserveSize = seqReserveSize;
-    }
-
-    /**
      * @return Swap enabled flag.
      */
     public boolean swapEnabled() {
@@ -355,13 +271,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param swapEnabled New swap enabled flag.
-     */
-    public void swapEnabled(boolean swapEnabled) {
-        this.swapEnabled = swapEnabled;
-    }
-
-    /**
      * @return Flag indicating whether Ignite should attempt to index value and/or key instances stored in cache.
      */
     public boolean queryIndexEnabled() {
@@ -369,14 +278,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param qryIdxEnabled New flag indicating whether Ignite should attempt to index value and/or key instances stored
-     * in cache.
-     */
-    public void queryIndexEnabled(boolean qryIdxEnabled) {
-        this.qryIdxEnabled = qryIdxEnabled;
-    }
-
-    /**
      * @return Invalidate.
      */
     public boolean invalidate() {
@@ -384,13 +285,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param invalidate New invalidate.
-     */
-    public void invalidate(boolean invalidate) {
-        this.invalidate = invalidate;
-    }
-
-    /**
      * @return Start size.
      */
     public int startSize() {
@@ -398,13 +292,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param startSize New start size.
-     */
-    public void startSize(int startSize) {
-        this.startSize = startSize;
-    }
-
-    /**
      * @return Name of class implementing GridCacheTmLookup.
      */
     @Nullable public String transactionManagerLookupClassName() {
@@ -412,13 +299,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param tmLookupClsName New name of class implementing GridCacheTmLookup.
-     */
-    public void transactionManagerLookupClassName(@Nullable String tmLookupClsName) {
-        this.tmLookupClsName = tmLookupClsName;
-    }
-
-    /**
      * @return Off-heap max memory.
      */
     public long offsetHeapMaxMemory() {
@@ -426,13 +306,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param offHeapMaxMemory New off-heap max memory.
-     */
-    public void offsetHeapMaxMemory(long offHeapMaxMemory) {
-        this.offHeapMaxMemory = offHeapMaxMemory;
-    }
-
-    /**
      * @return Max query iterator count
      */
     public int maxQueryIteratorCount() {
@@ -440,13 +313,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param maxQryIterCnt New max query iterator count
-     */
-    public void maxQueryIteratorCount(int maxQryIterCnt) {
-        this.maxQryIterCnt = maxQryIterCnt;
-    }
-
-    /**
      * @return Max concurrent async operations
      */
     public int maxConcurrentAsyncOperations() {
@@ -454,13 +320,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param maxConcurrentAsyncOps New max concurrent async operations
-     */
-    public void maxConcurrentAsyncOperations(int maxConcurrentAsyncOps) {
-        this.maxConcurrentAsyncOps = maxConcurrentAsyncOps;
-    }
-
-    /**
      * @return Memory mode.
      */
     public CacheMemoryMode memoryMode() {
@@ -468,13 +327,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param memoryMode New memory mode.
-     */
-    public void memoryMode(CacheMemoryMode memoryMode) {
-        this.memoryMode = memoryMode;
-    }
-
-    /**
      * @return Name of SPI to use for indexing.
      */
     public String indexingSpiName() {
@@ -482,13 +334,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param indexingSpiName New name of SPI to use for indexing.
-     */
-    public void indexingSpiName(String indexingSpiName) {
-        this.indexingSpiName = indexingSpiName;
-    }
-
-    /**
      * @return Cache interceptor.
      */
     @Nullable public String interceptor() {
@@ -496,116 +341,19 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param interceptor New cache interceptor.
-     */
-    public void interceptor(@Nullable String interceptor) {
-        this.interceptor = interceptor;
-    }
-
-    /**
-     * @return Cache affinity config.
-     */
-    public VisorCacheAffinityConfiguration affinityConfiguration() {
-        return affinity;
-    }
-
-    /**
-     * @param affinity New cache affinity config.
-     */
-    public void affinityConfiguration(VisorCacheAffinityConfiguration affinity) {
-        this.affinity = affinity;
-    }
-
-    /**
-     * @return Preload config.
-     */
-    public VisorCachePreloadConfiguration preloadConfiguration() {
-        return preload;
-    }
-
-    /**
-     * @param preload New preload config.
-     */
-    public void preloadConfiguration(VisorCachePreloadConfiguration preload) {
-        this.preload = preload;
-    }
-
-    /**
-     * @return Eviction config.
-     */
-    public VisorCacheEvictionConfiguration evictConfiguration() {
-        return evict;
-    }
-
-    /**
-     * @param evict New eviction config.
-     */
-    public void evictConfiguration(VisorCacheEvictionConfiguration evict) {
-        this.evict = evict;
-    }
-
-    /**
-     * @return Near cache config.
-     */
-    public VisorCacheNearConfiguration nearConfiguration() {
-        return near;
-    }
-
-    /**
-     * @param near New near cache config.
-     */
-    public void nearConfiguration(VisorCacheNearConfiguration near) {
-        this.near = near;
-    }
-
-    /**
-     * @return Dgc config
-     */
-    public VisorCacheDefaultConfiguration defaultConfiguration() {
-        return dflt;
-    }
-
-    /**
-     * @param dflt New default config
+     * @return Should value bytes be stored.
      */
-    public void defaultConfiguration(VisorCacheDefaultConfiguration dflt) {
-        this.dflt = dflt;
+    public boolean valueBytes() {
+        return valBytes;
     }
 
     /**
-     * @return Store config
+     * @param valBytes New should value bytes be stored.
      */
-    public VisorCacheStoreConfiguration storeConfiguration() {
-        return store;
+    public void valueBytes(boolean valBytes) {
+        this.valBytes = valBytes;
     }
 
-    /**
-     * @param store New store config
-     */
-    public void storeConfiguration(VisorCacheStoreConfiguration store) {
-        this.store = store;
-    }
-
-    /**
-     * @return Write behind config
-     */
-    public VisorCacheWriteBehindConfiguration writeBehind() {
-        return writeBehind;
-    }
-
-    /**
-     * @param writeBehind New write behind config
-     */
-    public void writeBehind(VisorCacheWriteBehindConfiguration writeBehind) {
-        this.writeBehind = writeBehind;
-    }
-
-    /**
-     * @param typeMeta New collection of type metadata.
-     */
-    public void typeMeta(Collection<VisorCacheTypeMetadata> typeMeta) {
-        this.typeMeta = typeMeta;
-    }
 
     /**
      * @return Collection of type metadata.
@@ -615,48 +363,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @return {@code true} if cache has JDBC store.
-     */
-    public boolean jdbcStore() {
-        return jdbcStore;
-    }
-
-    /**
-     * @param jdbcStore {@code true} if cache has JDBC store.
-     */
-    public void jdbcStore(boolean jdbcStore) {
-        this.jdbcStore = jdbcStore;
-    }
-
-    /**
-     * @return Whether cache should operate in read-through mode.
-     */
-    public boolean readThrough() {
-        return readThrough;
-    }
-
-    /**
-     * @param readThrough New whether cache should operate in read-through mode.
-     */
-    public void readThrough(boolean readThrough) {
-        this.readThrough = readThrough;
-    }
-
-    /**
-     * @return Whether cache should operate in write-through mode.
-     */
-    public boolean writeThrough() {
-        return writeThrough;
-    }
-
-    /**
-     * @param writeThrough New whether cache should operate in write-through mode.
-     */
-    public void writeThrough(boolean writeThrough) {
-        this.writeThrough = writeThrough;
-    }
-
-    /**
      * @return {@code true} if cache statistics enabled.
      */
     public boolean statisticsEnabled() {
@@ -664,71 +370,73 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @param statisticsEnabled {@code true} if cache statistics enabled.
+     * @return Whether management is enabled.
      */
-    public void statisticsEnabled(boolean statisticsEnabled) {
-        this.statisticsEnabled = statisticsEnabled;
+    public boolean managementEnabled() {
+        return mgmtEnabled;
     }
 
     /**
-     * @return Whether management is enabled.
+     * @return Class name of cache loader factory.
      */
-    public boolean managementEnabled() {
-        return mgmtEnabled;
+    public String loaderFactory() {
+        return ldrFactory;
     }
 
     /**
-     * @param mgmtEnabled New whether management is enabled.
+     * @return Class name of cache writer factory.
      */
-    public void managementEnabled(boolean mgmtEnabled) {
-        this.mgmtEnabled = mgmtEnabled;
+    public String writerFactory() {
+        return writerFactory;
     }
 
     /**
-     * @return Class name of cache loader factory.
+     * @return Class name of expiry policy factory.
      */
-    public String loaderFactory() {
-        return ldrFactory;
+    public String expiryPolicyFactory() {
+        return expiryPlcFactory;
     }
 
     /**
-     * @param ldrFactory New class name of cache loader factory.
+     * @return Cache affinityCfg config.
      */
-    public void loaderFactory(String ldrFactory) {
-        this.ldrFactory = ldrFactory;
+    public VisorCacheAffinityConfiguration affinityConfiguration() {
+        return affinityCfg;
     }
 
     /**
-     * @return Class name of cache writer factory.
+     * @return Preload config.
      */
-    public String writerFactory() {
-        return writerFactory;
+    public VisorCachePreloadConfiguration preloadConfiguration() {
+        return preloadCfg;
     }
 
     /**
-     * @param writerFactory New class name of cache writer factory.
+     * @return Eviction config.
      */
-    public void writerFactory(String writerFactory) {
-        this.writerFactory = writerFactory;
+    public VisorCacheEvictionConfiguration evictConfiguration() {
+        return evictCfg;
     }
 
     /**
-     * @return Class name of expiry policy factory.
+     * @return Near cache config.
      */
-    public String expiryPolicyFactory() {
-        return expiryPlcFactory;
+    public VisorCacheNearConfiguration nearConfiguration() {
+        return nearCfg;
     }
 
     /**
-     * @param expiryPlcFactory New class name of expiry policy factory.
+     * @return Dgc config
      */
-    public void expiryPolicyFactory(String expiryPlcFactory) {
-        this.expiryPlcFactory = expiryPlcFactory;
+    public VisorCacheDefaultConfiguration defaultConfiguration() {
+        return dfltCfg;
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheConfiguration.class, this);
+    /**
+     * @return Store config
+     */
+    public VisorCacheStoreConfiguration storeConfiguration() {
+        return storeCfg;
     }
 
     /**
@@ -738,10 +446,8 @@ public class VisorCacheConfiguration implements Serializable {
         return qryCfg;
     }
 
-    /**
-     * @param qryCfg New cache query configuration.
-     */
-    public void queryConfiguration(VisorCacheQueryConfiguration qryCfg) {
-        this.qryCfg = qryCfg;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheConfiguration.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/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
index 97f8f30..0996e65 100644
--- 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
@@ -45,9 +45,9 @@ public class VisorCacheDefaultConfiguration implements Serializable {
     public static VisorCacheDefaultConfiguration from(CacheConfiguration ccfg) {
         VisorCacheDefaultConfiguration cfg = new VisorCacheDefaultConfiguration();
 
-        cfg.timeToLive(ccfg.getDefaultTimeToLive());
-        cfg.txLockTimeout(ccfg.getDefaultLockTimeout());
-        cfg.queryTimeout(ccfg.getDefaultQueryTimeout());
+        cfg.ttl = ccfg.getDefaultTimeToLive();
+        cfg.txLockTimeout = ccfg.getDefaultLockTimeout();
+        cfg.qryTimeout = ccfg.getDefaultQueryTimeout();
 
         return cfg;
     }
@@ -60,13 +60,6 @@ public class VisorCacheDefaultConfiguration implements Serializable {
     }
 
     /**
-     * @param ttl New tTL value.
-     */
-    public void timeToLive(long ttl) {
-        this.ttl = ttl;
-    }
-
-    /**
      * @return Default transaction timeout.
      */
     public long txLockTimeout() {
@@ -74,26 +67,12 @@ public class VisorCacheDefaultConfiguration implements Serializable {
     }
 
     /**
-     * @param txLockTimeout New default transaction timeout.
-     */
-    public void txLockTimeout(long txLockTimeout) {
-        this.txLockTimeout = txLockTimeout;
-    }
-
-    /**
      * @return Default query timeout.
      */
     public long queryTimeout() {
         return qryTimeout;
     }
 
-    /**
-     * @param qryTimeout New default query timeout.
-     */
-    public void queryTimeout(long qryTimeout) {
-        this.qryTimeout = qryTimeout;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheDefaultConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/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 4b1a516..4c84f8d 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
@@ -69,15 +69,15 @@ public class VisorCacheEvictionConfiguration implements Serializable {
 
         final CacheEvictionPolicy plc = ccfg.getEvictionPolicy();
 
-        cfg.policy(compactClass(plc));
-        cfg.policyMaxSize(evictionPolicyMaxSize(plc));
-        cfg.filter(compactClass(ccfg.getEvictionFilter()));
-        cfg.synchronizedConcurrencyLevel(ccfg.getEvictSynchronizedConcurrencyLevel());
-        cfg.synchronizedTimeout(ccfg.getEvictSynchronizedTimeout());
-        cfg.synchronizedKeyBufferSize(ccfg.getEvictSynchronizedKeyBufferSize());
-        cfg.evictSynchronized(ccfg.isEvictSynchronized());
-        cfg.nearSynchronized(ccfg.isEvictNearSynchronized());
-        cfg.maxOverflowRatio(ccfg.getEvictMaxOverflowRatio());
+        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.nearSynchronized = ccfg.isEvictNearSynchronized();
+        cfg.maxOverflowRatio = ccfg.getEvictMaxOverflowRatio();
 
         return cfg;
     }
@@ -90,13 +90,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param plc New eviction policy.
-     */
-    public void policy(String plc) {
-        this.plc = plc;
-    }
-
-    /**
      * @return Cache eviction policy max size.
      */
     @Nullable public Integer policyMaxSize() {
@@ -104,13 +97,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param plcMaxSize New cache eviction policy max size.
-     */
-    public void policyMaxSize(Integer plcMaxSize) {
-        this.plcMaxSize = plcMaxSize;
-    }
-
-    /**
      * @return Eviction filter to specify which entries should not be evicted.
      */
     @Nullable public String filter() {
@@ -118,13 +104,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param filter New eviction filter to specify which entries should not be evicted.
-     */
-    public void filter(String filter) {
-        this.filter = filter;
-    }
-
-    /**
      * @return synchronized eviction concurrency level.
      */
     public int synchronizedConcurrencyLevel() {
@@ -132,13 +111,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param syncConcurrencyLvl New synchronized eviction concurrency level.
-     */
-    public void synchronizedConcurrencyLevel(int syncConcurrencyLvl) {
-        this.syncConcurrencyLvl = syncConcurrencyLvl;
-    }
-
-    /**
      * @return synchronized eviction timeout.
      */
     public long synchronizedTimeout() {
@@ -146,13 +118,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param syncTimeout New synchronized eviction timeout.
-     */
-    public void synchronizedTimeout(long syncTimeout) {
-        this.syncTimeout = syncTimeout;
-    }
-
-    /**
      * @return Synchronized key buffer size.
      */
     public int synchronizedKeyBufferSize() {
@@ -160,13 +125,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param syncKeyBufSize New synchronized key buffer size.
-     */
-    public void synchronizedKeyBufferSize(int syncKeyBufSize) {
-        this.syncKeyBufSize = syncKeyBufSize;
-    }
-
-    /**
      * @return Synchronous evicts flag.
      */
     public boolean evictSynchronized() {
@@ -174,13 +132,6 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param evictSynchronized New synchronous evicts flag.
-     */
-    public void evictSynchronized(boolean evictSynchronized) {
-        this.evictSynchronized = evictSynchronized;
-    }
-
-    /**
      * @return Synchronous near evicts flag.
      */
     public boolean nearSynchronized() {
@@ -188,26 +139,12 @@ public class VisorCacheEvictionConfiguration implements Serializable {
     }
 
     /**
-     * @param nearSynchronized New synchronous near evicts flag.
-     */
-    public void nearSynchronized(boolean nearSynchronized) {
-        this.nearSynchronized = nearSynchronized;
-    }
-
-    /**
      * @return Eviction max overflow ratio.
      */
     public float maxOverflowRatio() {
         return maxOverflowRatio;
     }
 
-    /**
-     * @param maxOverflowRatio New eviction max overflow ratio.
-     */
-    public void maxOverflowRatio(float maxOverflowRatio) {
-        this.maxOverflowRatio = maxOverflowRatio;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCacheEvictionConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/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 1f551a7..10db203 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
@@ -52,10 +52,10 @@ public class VisorCacheNearConfiguration implements Serializable {
     public static VisorCacheNearConfiguration from(CacheConfiguration ccfg) {
         VisorCacheNearConfiguration cfg = new VisorCacheNearConfiguration();
 
-        cfg.nearEnabled(GridCacheUtils.isNearEnabled(ccfg));
-        cfg.nearStartSize(ccfg.getNearStartSize());
-        cfg.nearEvictPolicy(compactClass(ccfg.getNearEvictionPolicy()));
-        cfg.nearEvictMaxSize(evictionPolicyMaxSize(ccfg.getNearEvictionPolicy()));
+        cfg.nearEnabled = GridCacheUtils.isNearEnabled(ccfg);
+        cfg.nearStartSize = ccfg.getNearStartSize();
+        cfg.nearEvictPlc = compactClass(ccfg.getNearEvictionPolicy());
+        cfg.nearEvictMaxSize = evictionPolicyMaxSize(ccfg.getNearEvictionPolicy());
 
         return cfg;
     }
@@ -68,13 +68,6 @@ public class VisorCacheNearConfiguration implements Serializable {
     }
 
     /**
-     * @param nearEnabled New flag to enable/disable near cache eviction policy.
-     */
-    public void nearEnabled(boolean nearEnabled) {
-        this.nearEnabled = nearEnabled;
-    }
-
-    /**
      * @return Near cache start size.
      */
     public int nearStartSize() {
@@ -82,13 +75,6 @@ public class VisorCacheNearConfiguration implements Serializable {
     }
 
     /**
-     * @param nearStartSize New near cache start size.
-     */
-    public void nearStartSize(int nearStartSize) {
-        this.nearStartSize = nearStartSize;
-    }
-
-    /**
      * @return Near cache eviction policy.
      */
     @Nullable public String nearEvictPolicy() {
@@ -96,28 +82,14 @@ public class VisorCacheNearConfiguration implements Serializable {
     }
 
     /**
-     * @param nearEvictPlc New near cache eviction policy.
-     */
-    public void nearEvictPolicy(String nearEvictPlc) {
-        this.nearEvictPlc = nearEvictPlc;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheNearConfiguration.class, this);
-    }
-
-    /**
      * @return Near cache eviction policy max size.
      */
     @Nullable public Integer nearEvictMaxSize() {
         return nearEvictMaxSize;
     }
 
-    /**
-     * @param nearEvictMaxSize New near cache eviction policy max size.
-     */
-    public void nearEvictMaxSize(@Nullable Integer nearEvictMaxSize) {
-        this.nearEvictMaxSize = nearEvictMaxSize;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheNearConfiguration.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadConfiguration.java
index 19a3734..f331789 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadConfiguration.java
@@ -55,12 +55,12 @@ public class VisorCachePreloadConfiguration implements Serializable {
     public static VisorCachePreloadConfiguration from(CacheConfiguration ccfg) {
         VisorCachePreloadConfiguration cfg = new VisorCachePreloadConfiguration();
 
-        cfg.mode(ccfg.getPreloadMode());
-        cfg.batchSize(ccfg.getPreloadBatchSize());
-        cfg.threadPoolSize(ccfg.getPreloadThreadPoolSize());
-        cfg.partitionedDelay(ccfg.getPreloadPartitionedDelay());
-        cfg.throttle(ccfg.getPreloadThrottle());
-        cfg.timeout(ccfg.getPreloadTimeout());
+        cfg.mode = ccfg.getPreloadMode();
+        cfg.batchSize = ccfg.getPreloadBatchSize();
+        cfg.threadPoolSize = ccfg.getPreloadThreadPoolSize();
+        cfg.partitionedDelay = ccfg.getPreloadPartitionedDelay();
+        cfg.throttle = ccfg.getPreloadThrottle();
+        cfg.timeout = ccfg.getPreloadTimeout();
 
         return cfg;
     }
@@ -73,13 +73,6 @@ public class VisorCachePreloadConfiguration implements Serializable {
     }
 
     /**
-     * @param mode New cache preload mode.
-     */
-    public void mode(CachePreloadMode mode) {
-        this.mode = mode;
-    }
-
-    /**
      * @return Preload thread pool size.
      */
     public int threadPoolSize() {
@@ -87,13 +80,6 @@ public class VisorCachePreloadConfiguration implements Serializable {
     }
 
     /**
-     * @param threadPoolSize New preload thread pool size.
-     */
-    public void threadPoolSize(int threadPoolSize) {
-        this.threadPoolSize = threadPoolSize;
-    }
-
-    /**
      * @return Cache preload batch size.
      */
     public int batchSize() {
@@ -101,13 +87,6 @@ public class VisorCachePreloadConfiguration implements Serializable {
     }
 
     /**
-     * @param batchSize New cache preload batch size.
-     */
-    public void batchSize(int batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    /**
      * @return Preloading partitioned delay.
      */
     public long partitionedDelay() {
@@ -115,13 +94,6 @@ public class VisorCachePreloadConfiguration implements Serializable {
     }
 
     /**
-     * @param partitionedDelay New preloading partitioned delay.
-     */
-    public void partitionedDelay(long partitionedDelay) {
-        this.partitionedDelay = partitionedDelay;
-    }
-
-    /**
      * @return Time in milliseconds to wait between preload messages.
      */
     public long throttle() {
@@ -129,26 +101,12 @@ public class VisorCachePreloadConfiguration implements Serializable {
     }
 
     /**
-     * @param throttle New time in milliseconds to wait between preload messages.
-     */
-    public void throttle(long throttle) {
-        this.throttle = throttle;
-    }
-
-    /**
      * @return Preload timeout.
      */
     public long timeout() {
         return timeout;
     }
 
-    /**
-     * @param timeout New preload timeout.
-     */
-    public void timeout(long timeout) {
-        this.timeout = timeout;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorCachePreloadConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
index b1ed170..7284b7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
@@ -17,7 +17,11 @@
 
 package org.apache.ignite.internal.visor.cache;
 
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -32,21 +36,58 @@ public class VisorCacheStoreConfiguration implements Serializable {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Cache store. */
+    /** Whether cache has JDBC store. */
+    private boolean jdbcStore;
+
+    /** Cache store class name. */
     private String store;
 
-    /** Should value bytes be stored. */
-    private boolean valBytes;
+    /** Cache store factory class name. */
+    private String storeFactory;
+
+    /** Whether cache should operate in read-through mode. */
+    private boolean readThrough;
+
+    /** Whether cache should operate in write-through mode. */
+    private boolean writeThrough;
+
+    /** Flag indicating whether write-behind behaviour should be used for the cache store. */
+    private boolean writeBehindEnabled;
+
+    /** Maximum batch size for write-behind cache store operations. */
+    private int batchSz;
+
+    /** Frequency with which write-behind cache is flushed to the cache store in milliseconds. */
+    private long flushFreq;
+
+    /** Maximum object count in write-behind cache. */
+    private int flushSz;
+
+    /** Number of threads that will perform cache flushing. */
+    private int flushThreadCnt;
 
     /**
      * @param ccfg Cache configuration.
      * @return Data transfer object for cache store configuration properties.
      */
-    public static VisorCacheStoreConfiguration from(CacheConfiguration ccfg) {
+    public static VisorCacheStoreConfiguration from(Ignite ignite, CacheConfiguration ccfg) {
         VisorCacheStoreConfiguration cfg = new VisorCacheStoreConfiguration();
 
-        cfg.store(compactClass(ccfg.getCacheStoreFactory()));
-        cfg.valueBytes(ccfg.isStoreValueBytes());
+        CacheStore store = ((IgniteKernal)ignite).internalCache(ccfg.getName()).context().store().configuredStore();
+
+        cfg.jdbcStore = store instanceof CacheAbstractJdbcStore;
+
+        cfg.store = compactClass(store);
+        cfg.storeFactory = compactClass(ccfg.getCacheStoreFactory());
+
+        cfg.readThrough = ccfg.isReadThrough();
+        cfg.writeThrough = ccfg.isWriteThrough();
+
+        cfg.writeBehindEnabled = ccfg.isWriteBehindEnabled();
+        cfg.batchSz = ccfg.getWriteBehindBatchSize();
+        cfg.flushFreq = ccfg.getWriteBehindFlushFrequency();
+        cfg.flushSz = ccfg.getWriteBehindFlushSize();
+        cfg.flushThreadCnt = ccfg.getWriteBehindFlushThreadCount();
 
         return cfg;
     }
@@ -59,6 +100,13 @@ public class VisorCacheStoreConfiguration implements Serializable {
     }
 
     /**
+     * @return {@code true} if cache has JDBC store.
+     */
+    public boolean jdbcStore() {
+        return jdbcStore;
+    }
+
+    /**
      * @return Cache store class name.
      */
     @Nullable public String store() {
@@ -66,24 +114,94 @@ public class VisorCacheStoreConfiguration implements Serializable {
     }
 
     /**
-     * @param store Cache store class name.
+     * @return Cache store factory class name..
+     */
+    public String storeFactory() {
+        return storeFactory;
+    }
+
+    /**
+     * @return Whether cache should operate in read-through mode.
+     */
+    public boolean readThrough() {
+        return readThrough;
+    }
+
+    /**
+     * @return Whether cache should operate in write-through mode.
+     */
+    public boolean writeThrough() {
+        return writeThrough;
+    }
+
+    /**
+     * @return Flag indicating whether write-behind behaviour should be used for the cache store.
+     */
+    public boolean writeBehindEnabled() {
+        return writeBehindEnabled;
+    }
+
+    /**
+     * @param writeBehindEnabled New flag indicating whether write-behind behaviour should be used for the cache store.
+     */
+    public void writeBehindEnabled(boolean writeBehindEnabled) {
+        this.writeBehindEnabled = writeBehindEnabled;
+    }
+
+    /**
+     * @return Maximum batch size for write-behind cache store operations.
+     */
+    public int batchSize() {
+        return batchSz;
+    }
+
+    /**
+     * @param batchSize New maximum batch size for write-behind cache store operations.
+     */
+    public void batchSize(int batchSize) {
+        this.batchSz = batchSize;
+    }
+
+    /**
+     * @return Frequency with which write-behind cache is flushed to the cache store in milliseconds.
+     */
+    public long flushFrequency() {
+        return flushFreq;
+    }
+
+    /**
+     * @param flushFreq New frequency with which write-behind cache is flushed to the cache store in milliseconds.
+     */
+    public void flushFrequency(long flushFreq) {
+        this.flushFreq = flushFreq;
+    }
+
+    /**
+     * @return Maximum object count in write-behind cache.
+     */
+    public int flushSize() {
+        return flushSz;
+    }
+
+    /**
+     * @param flushSize New maximum object count in write-behind cache.
      */
-    public void store(String store) {
-        this.store = store;
+    public void flushSize(int flushSize) {
+        this.flushSz = flushSize;
     }
 
     /**
-     * @return Should value bytes be stored.
+     * @return Number of threads that will perform cache flushing.
      */
-    public boolean valueBytes() {
-        return valBytes;
+    public int flushThreadCount() {
+        return flushThreadCnt;
     }
 
     /**
-     * @param valBytes New should value bytes be stored.
+     * @param flushThreadCnt New number of threads that will perform cache flushing.
      */
-    public void valueBytes(boolean valBytes) {
-        this.valBytes = valBytes;
+    public void flushThreadCount(int flushThreadCnt) {
+        this.flushThreadCnt = flushThreadCnt;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheWriteBehindConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheWriteBehindConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheWriteBehindConfiguration.java
deleted file mode 100644
index c55078c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheWriteBehindConfiguration.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.visor.cache;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Data transfer object for write-behind cache configuration properties.
- */
-public class VisorCacheWriteBehindConfiguration implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Flag indicating whether write-behind behaviour should be used for the cache store. */
-    private boolean enabled;
-
-    /** Maximum batch size for write-behind cache store operations. */
-    private int batchSize;
-
-    /** Frequency with which write-behind cache is flushed to the cache store in milliseconds. */
-    private long flushFreq;
-
-    /** Maximum object count in write-behind cache. */
-    private int flushSize;
-
-    /** Number of threads that will perform cache flushing. */
-    private int flushThreadCnt;
-
-    /**
-     * @param ccfg Cache configuration.
-     * @return Data transfer object for write-behind cache configuration properties.
-     */
-    public static VisorCacheWriteBehindConfiguration from(CacheConfiguration ccfg) {
-        VisorCacheWriteBehindConfiguration cfg = new VisorCacheWriteBehindConfiguration();
-
-        cfg.enabled(ccfg.isWriteBehindEnabled());
-        cfg.batchSize(ccfg.getWriteBehindBatchSize());
-        cfg.flushFrequency(ccfg.getWriteBehindFlushFrequency());
-        cfg.flushSize(ccfg.getWriteBehindFlushSize());
-        cfg.flushThreadCount(ccfg.getWriteBehindFlushThreadCount());
-
-        return cfg;
-    }
-
-    /**
-     * @return Flag indicating whether write-behind behaviour should be used for the cache store.
-     */
-    public boolean enabled() {
-        return enabled;
-    }
-
-    /**
-     * @param enabled New flag indicating whether write-behind behaviour should be used for the cache store.
-     */
-    public void enabled(boolean enabled) {
-        this.enabled = enabled;
-    }
-
-    /**
-     * @return Maximum batch size for write-behind cache store operations.
-     */
-    public int batchSize() {
-        return batchSize;
-    }
-
-    /**
-     * @param batchSize New maximum batch size for write-behind cache store operations.
-     */
-    public void batchSize(int batchSize) {
-        this.batchSize = batchSize;
-    }
-
-    /**
-     * @return Frequency with which write-behind cache is flushed to the cache store in milliseconds.
-     */
-    public long flushFrequency() {
-        return flushFreq;
-    }
-
-    /**
-     * @param flushFreq New frequency with which write-behind cache is flushed to the cache store in milliseconds.
-     */
-    public void flushFrequency(long flushFreq) {
-        this.flushFreq = flushFreq;
-    }
-
-    /**
-     * @return Maximum object count in write-behind cache.
-     */
-    public int flushSize() {
-        return flushSize;
-    }
-
-    /**
-     * @param flushSize New maximum object count in write-behind cache.
-     */
-    public void flushSize(int flushSize) {
-        this.flushSize = flushSize;
-    }
-
-    /**
-     * @return Number of threads that will perform cache flushing.
-     */
-    public int flushThreadCount() {
-        return flushThreadCnt;
-    }
-
-    /**
-     * @param flushThreadCnt New number of threads that will perform cache flushing.
-     */
-    public void flushThreadCount(int flushThreadCnt) {
-        this.flushThreadCnt = flushThreadCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorCacheWriteBehindConfiguration.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index a9b4e57..b03674f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -229,7 +229,7 @@ public class VisorTaskUtils {
      * @param obj Object for compact.
      * @return Compacted string.
      */
-    @Nullable public static String compactClass(Object obj) {
+    @Nullable public static String compactClass(@Nullable Object obj) {
         if (obj == null)
             return null;
 
@@ -613,7 +613,7 @@ public class VisorTaskUtils {
      * @param plc Eviction policy.
      * @return Extracted max size.
      */
-    public static Integer evictionPolicyMaxSize(CacheEvictionPolicy plc) {
+    public static Integer evictionPolicyMaxSize(@Nullable CacheEvictionPolicy plc) {
         if (plc instanceof CacheLruEvictionPolicyMBean)
             return ((CacheLruEvictionPolicyMBean)plc).getMaxSize();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ff0c1e1d/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 3c1aa01..97adeac 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -776,7 +776,6 @@ object VisorCacheCommand {
         val evictCfg = cfg.evictConfiguration()
         val defaultCfg = cfg.defaultConfiguration()
         val storeCfg = cfg.storeConfiguration()
-        val writeBehind = cfg.writeBehind()
 
         val cacheT = VisorTextTable()
 
@@ -836,20 +835,25 @@ object VisorCacheCommand {
         cacheT += ("Indexing SPI Name", cfg.indexingSpiName())
         cacheT += ("Cache Interceptor", cfg.interceptor())
 
-        cacheT += ("Store Enabled", storeCfg.enabled())
-        cacheT += ("Store", storeCfg.store())
-        cacheT += ("Store Values In Bytes", storeCfg.valueBytes())
+        cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
+        cacheT += ("Memory Mode", cfg.memoryMode())
+
+        cacheT += ("Store Values In Bytes", cfg.valueBytes())
 
         cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
 
-        cacheT += ("Write-Behind Enabled", writeBehind.enabled())
-        cacheT += ("Write-Behind Flush Size", writeBehind.flushSize())
-        cacheT += ("Write-Behind Frequency", writeBehind.flushFrequency())
-        cacheT += ("Write-Behind Flush Threads Count", writeBehind.flushThreadCount())
-        cacheT += ("Write-Behind Batch Size", writeBehind.batchSize())
+        cacheT += ("Store Enabled", storeCfg.enabled())
+        cacheT += ("Store", storeCfg.store())
+        cacheT += ("Store Factory", storeCfg.storeFactory())
 
-        cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
-        cacheT += ("Memory Mode", cfg.memoryMode())
+        cacheT += ("Store Read-Through", storeCfg.readThrough())
+        cacheT += ("Store Write-Through", storeCfg.writeThrough())
+
+        cacheT += ("Write-Behind Enabled", storeCfg.writeBehindEnabled())
+        cacheT += ("Write-Behind Flush Size", storeCfg.flushSize())
+        cacheT += ("Write-Behind Frequency", storeCfg.flushFrequency())
+        cacheT += ("Write-Behind Flush Threads Count", storeCfg.flushThreadCount())
+        cacheT += ("Write-Behind Batch Size", storeCfg.batchSize())
 
         println(title)
 


[2/8] incubator-ignite git commit: Merge branch 'sprint-2' of http://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339

Posted by ak...@apache.org.
Merge branch 'sprint-2' of http://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339


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

Branch: refs/heads/sprint-2
Commit: 53141e550eb8af27bcf9b5d0d6fe73c465150130
Parents: ff0c1e1 47539d8
Author: anovikov <an...@gridgain.com>
Authored: Fri Feb 27 15:50:02 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Fri Feb 27 15:50:02 2015 +0700

----------------------------------------------------------------------

----------------------------------------------------------------------



[7/8] incubator-ignite git commit: Merge branches 'ignite-339' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339

Posted by ak...@apache.org.
Merge branches 'ignite-339' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339


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

Branch: refs/heads/sprint-2
Commit: 60e0ecb2e269e6af2b0c04df2aed6f494657b03c
Parents: 5d3a2be 13e2d1f
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Mar 2 15:59:00 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Mar 2 15:59:00 2015 +0700

----------------------------------------------------------------------
 .../store/CacheNodeWithStoreStartup.java        |   6 +-
 .../store/jdbc/CacheJdbcPersonStore.java        | 115 ++++++----------
 .../apache/ignite/cache/store/CacheStore.java   |   4 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |   2 +-
 .../ignite/cache/store/CacheStoreSession.java   |  17 ++-
 .../processors/cache/GridCacheStoreManager.java |   6 +-
 .../ignite/internal/util/IgniteUtils.java       |  90 ++++++++++--
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 137 +++++++++++--------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 116 ++++++++++++++++
 .../core/src/test/config/store/jdbc/Ignite.xml  |  63 +++++++--
 .../junits/cache/TestCacheSession.java          |   5 +
 .../cache/TestThreadLocalCacheSession.java      |   5 +
 .../ignite/schema/generator/XmlGenerator.java   |   8 +-
 .../apache/ignite/schema/model/PojoField.java   |  11 +-
 .../apache/ignite/schema/load/model/Ignite.xml  | 133 +++++++++++++-----
 .../yardstick/config/ignite-store-config.xml    |  15 +-
 16 files changed, 521 insertions(+), 212 deletions(-)
----------------------------------------------------------------------



[5/8] incubator-ignite git commit: Merge branches 'ignite-339' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339

Posted by ak...@apache.org.
Merge branches 'ignite-339' and 'sprint-2' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-339


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

Branch: refs/heads/sprint-2
Commit: 2710ece0fb8c18fec65d7e4f745f048eca07790e
Parents: 57bdd8a a792c99
Author: AKuznetsov <ak...@gridgain.com>
Authored: Sat Feb 28 01:40:08 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Sat Feb 28 01:40:08 2015 +0700

----------------------------------------------------------------------
 config/hadoop/default-config.xml                |   12 +
 config/ignite-log4j.xml                         |    2 +-
 .../datagrid/CacheContinuousQueryExample.java   |    2 +-
 ipc/shmem/Makefile.am                           |   15 +
 ipc/shmem/igniteshmem/Makefile.am               |   15 +
 ipc/shmem/include/Makefile.am                   |   15 +
 modules/clients/src/test/keystore/generate.sh   |   15 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    4 +-
 .../ignite/cache/query/ContinuousQuery.java     |   18 +-
 .../apache/ignite/cluster/ClusterMetrics.java   |    2 +-
 .../configuration/QueryConfiguration.java       |   37 +-
 .../java/org/apache/ignite/igfs/IgfsMode.java   |    6 +-
 .../apache/ignite/internal/GridProperties.java  |   78 -
 .../ignite/internal/GridUpdateNotifier.java     |    2 +-
 .../apache/ignite/internal/IgniteKernal.java    |   18 +-
 .../ignite/internal/IgniteProperties.java       |   79 +
 .../ignite/internal/IgniteVersionUtils.java     |    8 +-
 .../internal/events/DiscoveryCustomEvent.java   |    3 +
 .../processors/cache/IgniteCacheProxy.java      |    6 +-
 .../plugin/IgnitePluginProcessor.java           |   24 +
 .../optimized-classnames.previous.properties    |   15 +
 .../optimized/optimized-classnames.properties   | 1565 +-----------------
 .../apache/ignite/plugin/PluginProvider.java    |    5 +
 .../TcpDiscoveryCustomEventMessage.java         |    3 +
 .../internal/GridUpdateNotifierSelfTest.java    |    2 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 .../config/GridTestProperties.java              |   10 +-
 modules/extdata/p2p/pom.xml                     |    6 -
 .../client/hadoop/GridHadoopClientProtocol.java |    6 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  412 +++++
 .../igfs/hadoop/v1/IgfsHadoopFileSystem.java    |    3 +-
 .../igfs/hadoop/v2/IgfsHadoopFileSystem.java    |    3 +-
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |   10 +-
 .../hadoop/IgfsHadoopFileSystemWrapper.java     |  413 -----
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |    2 +-
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |    4 +-
 .../hadoop/GridHadoopClassLoader.java           |   12 +-
 .../processors/hadoop/GridHadoopSetup.java      |    8 +-
 .../processors/hadoop/GridHadoopUtils.java      |    4 +-
 .../collections/GridHadoopHashMultimapBase.java |    2 +-
 .../GridHadoopExternalCommunication.java        |   14 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |    6 +-
 .../v2/GridHadoopV2JobResourceManager.java      |    2 +-
 .../GridHadoopClientProtocolSelfTest.java       |    6 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |    2 +-
 .../IgfsHadoop20FileSystemAbstractSelfTest.java |    2 +-
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |    2 +-
 .../IgfsHadoopFileSystemAbstractSelfTest.java   |    1 +
 ...fsHadoopFileSystemSecondaryModeSelfTest.java |    2 +-
 .../hadoop/GridHadoopGroupingTest.java          |    4 +-
 .../igfs/IgfsPerformanceBenchmark.java          |    9 +-
 modules/hibernate/pom.xml                       |    6 -
 .../HibernateReadWriteAccessStrategy.java       |   81 +-
 modules/indexing/pom.xml                        |    6 -
 modules/jta/pom.xml                             |    6 -
 modules/scalar/pom.xml                          |    6 -
 modules/spring/pom.xml                          |    6 -
 modules/visor-console/pom.xml                   |    7 -
 modules/web/pom.xml                             |    6 -
 modules/winservice/IgniteService.sln            |    2 +-
 .../IgniteService/IgniteService.csproj          |    2 +-
 .../config/benchmark-atomic-win.properties      |   15 +
 .../config/benchmark-atomic.properties          |   15 +
 .../config/benchmark-compute-win.properties     |   15 +
 .../config/benchmark-compute.properties         |   15 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-query-win.properties       |   15 +
 .../yardstick/config/benchmark-query.properties |   15 +
 .../config/benchmark-tx-win.properties          |   15 +
 .../yardstick/config/benchmark-tx.properties    |   15 +
 .../yardstick/config/benchmark-win.properties   |   15 +
 modules/yardstick/config/benchmark.properties   |   15 +
 pom.xml                                         |  150 +-
 73 files changed, 1074 insertions(+), 2273 deletions(-)
----------------------------------------------------------------------



[8/8] incubator-ignite git commit: # IGNITE-339 Review.

Posted by ak...@apache.org.
# IGNITE-339 Review.


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

Branch: refs/heads/sprint-2
Commit: c847e88513d47925a03e84faf8739b6e7b012755
Parents: 60e0ecb
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Mar 2 16:44:13 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Mar 2 16:44:13 2015 +0700

----------------------------------------------------------------------
 .../internal/visor/cache/VisorCacheConfiguration.java | 10 ----------
 .../visor/commands/cache/VisorCacheCommand.scala      | 14 +++++---------
 2 files changed, 5 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c847e885/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 2ad0e49..a6ec05c 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
@@ -48,9 +48,6 @@ public class VisorCacheConfiguration implements Serializable {
     /** Cache atomicity mode */
     private CacheAtomicityMode atomicityMode;
 
-    /** Cache atomic sequence reserve size */
-    private int atomicSeqReserveSize;
-
     /** Cache atomicity write ordering mode. */
     private CacheAtomicWriteOrderMode atomicWriteOrderMode;
 
@@ -226,13 +223,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @return Cache atomic sequence reserve size
-     */
-    public int atomicSequenceReserveSize() {
-        return atomicSeqReserveSize;
-    }
-
-    /**
      * @return Cache atomicity write ordering mode.
      */
     public CacheAtomicWriteOrderMode atomicWriteOrderMode() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c847e885/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 7232221..e2ca05b 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -783,7 +783,6 @@ object VisorCacheCommand {
 
         cacheT += ("Mode", cfg.mode)
         cacheT += ("Atomicity Mode", safe(cfg.atomicityMode))
-        cacheT += ("Atomic Sequence Reserve Size", cfg.atomicSequenceReserveSize)
         cacheT += ("Atomic Write Ordering Mode", safe(cfg.atomicWriteOrderMode))
         cacheT += ("Statistic Enabled", bool2Str(cfg.statisticsEnabled()))
         cacheT += ("Management Enabled", bool2Str(cfg.managementEnabled()))
@@ -839,11 +838,10 @@ object VisorCacheCommand {
         cacheT += ("Cache Interceptor", safe(cfg.interceptor()))
 
         cacheT += ("Store Enabled", bool2Str(storeCfg.enabled()))
-        cacheT += ("Store", safe(storeCfg.store()))
-        cacheT += ("Configured JDBC Store", bool2Str(storeCfg.jdbcStore()))
-
-        cacheT += ("Read Through", bool2Str(storeCfg.readThrough()))
-        cacheT += ("Write Through", bool2Str(storeCfg.writeThrough()))
+        cacheT += ("Store Сlass", safe(storeCfg.store()))
+        cacheT += ("Store Factory Сlass", storeCfg.storeFactory())
+        cacheT += ("Store Read Through", bool2Str(storeCfg.readThrough()))
+        cacheT += ("Store Write Through", bool2Str(storeCfg.writeThrough()))
 
         cacheT += ("Write-Behind Enabled", bool2Str(storeCfg.enabled()))
         cacheT += ("Write-Behind Flush Size", storeCfg.flushSize())
@@ -853,9 +851,7 @@ object VisorCacheCommand {
 
         cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
         cacheT += ("Memory Mode", cfg.memoryMode())
-
-        cacheT += ("Store Values Bytes", cfg.valueBytes())
-
+        cacheT += ("Keep Values Bytes", cfg.valueBytes())
         cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
 
         cacheT += ("Loader Factory Class Name", safe(cfg.loaderFactory()))


[3/8] incubator-ignite git commit: # Merge sprint-2 into ignite-339

Posted by ak...@apache.org.
# Merge sprint-2 into ignite-339


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

Branch: refs/heads/sprint-2
Commit: 555e3f9455dc82a011943a561c0aa6405cca82bc
Parents: 53141e5 4e7463d
Author: anovikov <an...@gridgain.com>
Authored: Fri Feb 27 17:21:33 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Fri Feb 27 17:21:33 2015 +0700

----------------------------------------------------------------------
 .../visor/node/VisorBasicConfiguration.java     |  17 ---
 .../commands/alert/VisorAlertCommand.scala      |   8 +-
 .../commands/cache/VisorCacheCommand.scala      |  95 +++++++------
 .../config/VisorConfigurationCommand.scala      | 140 ++++++++++---------
 .../commands/disco/VisorDiscoveryCommand.scala  |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  64 ++++++---
 .../commands/tasks/VisorTasksCommandSpec.scala  |   2 +-
 7 files changed, 178 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/555e3f94/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --cc modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 97adeac,5dd19b1..577067a
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@@ -776,6 -775,8 +775,7 @@@ object VisorCacheCommand 
          val evictCfg = cfg.evictConfiguration()
          val defaultCfg = cfg.defaultConfiguration()
          val storeCfg = cfg.storeConfiguration()
 -        val writeBehind = cfg.writeBehind()
+         val queryCfg = cfg.queryConfiguration()
  
          val cacheT = VisorTextTable()
  
@@@ -789,19 -792,21 +791,19 @@@
          cacheT += ("Time To Live", defaultCfg.timeToLive())
          cacheT += ("Time To Live Eager Flag", cfg.eagerTtl)
  
-         cacheT += ("Write Synchronization Mode", cfg.writeSynchronizationMode)
-         cacheT += ("Swap Enabled", cfg.swapEnabled())
-         cacheT += ("Invalidate", cfg.invalidate())
+         cacheT += ("Write Synchronization Mode", safe(cfg.writeSynchronizationMode))
+         cacheT += ("Swap Enabled", bool2Str(cfg.swapEnabled()))
+         cacheT += ("Invalidate", bool2Str(cfg.invalidate()))
 -        cacheT += ("Read Through", bool2Str(cfg.readThrough()))
 -        cacheT += ("Write Through", bool2Str(cfg.writeThrough()))
          cacheT += ("Start Size", cfg.startSize())
  
-         cacheT += ("Transaction Manager Lookup", cfg.transactionManagerLookupClassName())
+         cacheT += ("Transaction Manager Lookup", safe(cfg.transactionManagerLookupClassName()))
  
-         cacheT += ("Affinity Function", affinityCfg.function())
+         cacheT += ("Affinity Function", safe(affinityCfg.function()))
          cacheT += ("Affinity Backups", affinityCfg.partitionedBackups())
-         cacheT += ("Affinity Partitions", affinityCfg.partitions())
-         cacheT += ("Affinity Default Replicas", affinityCfg.defaultReplicas())
-         cacheT += ("Affinity Exclude Neighbors", affinityCfg.excludeNeighbors())
-         cacheT += ("Affinity Mapper", affinityCfg.mapper())
+         cacheT += ("Affinity Partitions", safe(affinityCfg.partitions()))
+         cacheT += ("Affinity Default Replicas", safe(affinityCfg.defaultReplicas()))
+         cacheT += ("Affinity Exclude Neighbors", safe(affinityCfg.excludeNeighbors()))
+         cacheT += ("Affinity Mapper", safe(affinityCfg.mapper()))
  
          cacheT += ("Preload Mode", preloadCfg.mode())
          cacheT += ("Preload Batch Size", preloadCfg.batchSize())
@@@ -830,30 -835,41 +832,45 @@@
  
          cacheT += ("Default Lock Timeout", defaultCfg.txLockTimeout())
          cacheT += ("Default Query Timeout", defaultCfg.queryTimeout())
-         cacheT += ("Query Indexing Enabled", cfg.queryIndexEnabled())
+         cacheT += ("Query Indexing Enabled", bool2Str(cfg.queryIndexEnabled()))
          cacheT += ("Query Iterators Number", cfg.maxQueryIteratorCount())
-         cacheT += ("Indexing SPI Name", cfg.indexingSpiName())
-         cacheT += ("Cache Interceptor", cfg.interceptor())
+         cacheT += ("Metadata type count", cfg.typeMeta().size())
+         cacheT += ("Indexing SPI Name", safe(cfg.indexingSpiName()))
+         cacheT += ("Cache Interceptor", safe(cfg.interceptor()))
+ 
+         cacheT += ("Store Enabled", bool2Str(storeCfg.enabled()))
+         cacheT += ("Store", safe(storeCfg.store()))
 -        cacheT += ("Store Values In Bytes", storeCfg.valueBytes())
 -        cacheT += ("Configured JDBC Store", bool2Str(cfg.jdbcStore()))
++        cacheT += ("Configured JDBC Store", bool2Str(storeCfg.jdbcStore()))
+ 
 -        cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
++        cacheT += ("Read Through", bool2Str(storeCfg.readThrough()))
++        cacheT += ("Write Through", bool2Str(storeCfg.writeThrough()))
+ 
 -        cacheT += ("Write-Behind Enabled", bool2Str(writeBehind.enabled()))
 -        cacheT += ("Write-Behind Flush Size", writeBehind.flushSize())
 -        cacheT += ("Write-Behind Frequency", writeBehind.flushFrequency())
 -        cacheT += ("Write-Behind Flush Threads Count", writeBehind.flushThreadCount())
 -        cacheT += ("Write-Behind Batch Size", writeBehind.batchSize())
++        cacheT += ("Write-Behind Enabled", bool2Str(storeCfg.enabled()))
++        cacheT += ("Write-Behind Flush Size", storeCfg.flushSize())
++        cacheT += ("Write-Behind Frequency", storeCfg.flushFrequency())
++        cacheT += ("Write-Behind Flush Threads Count", storeCfg.flushThreadCount())
++        cacheT += ("Write-Behind Batch Size", storeCfg.batchSize())
  
          cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
          cacheT += ("Memory Mode", cfg.memoryMode())
  
 +        cacheT += ("Store Values In Bytes", cfg.valueBytes())
 +
 +        cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())
 +
-         cacheT += ("Store Enabled", storeCfg.enabled())
-         cacheT += ("Store", storeCfg.store())
-         cacheT += ("Store Factory", storeCfg.storeFactory())
- 
-         cacheT += ("Store Read-Through", storeCfg.readThrough())
-         cacheT += ("Store Write-Through", storeCfg.writeThrough())
+         cacheT += ("Loader Factory Class Name", safe(cfg.loaderFactory()))
+         cacheT += ("Writer Factory Class Name", safe(cfg.writerFactory()))
+         cacheT += ("Expiry Policy Factory Class Name", safe(cfg.expiryPolicyFactory()))
  
-         cacheT += ("Write-Behind Enabled", storeCfg.writeBehindEnabled())
-         cacheT += ("Write-Behind Flush Size", storeCfg.flushSize())
-         cacheT += ("Write-Behind Frequency", storeCfg.flushFrequency())
-         cacheT += ("Write-Behind Flush Threads Count", storeCfg.flushThreadCount())
-         cacheT += ("Write-Behind Batch Size", storeCfg.batchSize())
+         if (queryCfg != null) {
+             cacheT +=("Query Type Resolver", safe(queryCfg.typeResolver()))
+             cacheT +=("Query Indexing Primitive Key", bool2Str(queryCfg.indexPrimitiveKey()))
+             cacheT +=("Query Indexing Primitive Value", bool2Str(queryCfg.indexPrimitiveValue()))
+             cacheT +=("Query Fixed Typing", bool2Str(queryCfg.indexFixedTyping()))
+             cacheT +=("Query Escaped Names", bool2Str(queryCfg.escapeAll()))
+         }
+         else
+             cacheT += ("Query Configuration", NA)
  
          println(title)
  


[4/8] incubator-ignite git commit: # ignite-339 Refactored Visor Configurations

Posted by ak...@apache.org.
# ignite-339 Refactored Visor Configurations


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

Branch: refs/heads/sprint-2
Commit: 57bdd8a5412bec7fafd2481bc42752d50de36463
Parents: 555e3f9
Author: anovikov <an...@gridgain.com>
Authored: Fri Feb 27 18:20:57 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Fri Feb 27 18:20:57 2015 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheConfiguration.java    |  10 -
 .../visor/node/VisorAtomicConfiguration.java    |  27 +-
 .../visor/node/VisorBasicConfiguration.java     | 180 ++------------
 .../node/VisorCacheQueryConfiguration.java      |  45 +---
 .../node/VisorExecutorServiceConfiguration.java |  54 +---
 .../visor/node/VisorGridConfiguration.java      | 177 ++------------
 .../visor/node/VisorIgfsConfiguration.java      | 244 ++-----------------
 .../visor/node/VisorLifecycleConfiguration.java |   9 +-
 .../visor/node/VisorMetricsConfiguration.java   |  29 +--
 .../node/VisorPeerToPeerConfiguration.java      |  28 +--
 .../visor/node/VisorQueryConfiguration.java     |  65 +----
 .../visor/node/VisorRestConfiguration.java      |  80 +-----
 .../node/VisorSegmentationConfiguration.java    |  45 +---
 .../visor/node/VisorSpisConfiguration.java      |  92 +------
 .../node/VisorTransactionConfiguration.java     |  64 +----
 15 files changed, 139 insertions(+), 1010 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 cf149f7..8eee437 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
@@ -60,9 +60,6 @@ public class VisorCacheConfiguration implements Serializable {
     /** Write synchronization mode. */
     private CacheWriteSynchronizationMode writeSynchronizationMode;
 
-    /** Sequence reserve size. */
-    private int seqReserveSize;
-
     /** Swap enabled flag. */
     private boolean swapEnabled;
 
@@ -257,13 +254,6 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @return Sequence reserve size.
-     */
-    public int sequenceReserveSize() {
-        return seqReserveSize;
-    }
-
-    /**
      * @return Swap enabled flag.
      */
     public boolean swapEnabled() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 79ce903..c39e3d9 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
@@ -48,9 +48,9 @@ public class VisorAtomicConfiguration implements Serializable {
     public static VisorAtomicConfiguration from(AtomicConfiguration src) {
         VisorAtomicConfiguration cfg = new VisorAtomicConfiguration();
 
-        cfg.atomicSequenceReserveSize(src.getAtomicSequenceReserveSize());
-        cfg.cacheMode(src.getCacheMode());
-        cfg.backups(src.getBackups());
+        cfg.seqReserveSize = src.getAtomicSequenceReserveSize();
+        cfg.cacheMode = src.getCacheMode();
+        cfg.backups = src.getBackups();
 
         return cfg;
     }
@@ -63,13 +63,6 @@ public class VisorAtomicConfiguration implements Serializable {
     }
 
     /**
-     * @param seqReserveSize Atomic sequence reservation size.
-     */
-    public void atomicSequenceReserveSize(int seqReserveSize) {
-        this.seqReserveSize = seqReserveSize;
-    }
-
-    /**
      * @return Cache mode.
      */
     public CacheMode cacheMode() {
@@ -77,26 +70,12 @@ public class VisorAtomicConfiguration implements Serializable {
     }
 
     /**
-     * @param cacheMode Cache mode.
-     */
-    public void cacheMode(CacheMode cacheMode) {
-        this.cacheMode = cacheMode;
-    }
-
-    /**
      * @return Number of backup nodes.
      */
     public int backups() {
         return backups;
     }
 
-    /**
-     * @param backups Number of backup nodes.
-     */
-    public void backups(int backups) {
-        this.backups = backups;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorAtomicConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 8a79299..a3b6052 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
@@ -104,26 +104,26 @@ public class VisorBasicConfiguration implements Serializable {
     public static VisorBasicConfiguration from(IgniteEx ignite, IgniteConfiguration c) {
         VisorBasicConfiguration cfg = new VisorBasicConfiguration();
 
-        cfg.gridName(c.getGridName());
-        cfg.ggHome(getProperty(IGNITE_HOME, c.getIgniteHome()));
-        cfg.localHost(getProperty(IGNITE_LOCAL_HOST, c.getLocalHost()));
-        cfg.nodeId(ignite.localNode().id());
-        cfg.marshaller(compactClass(c.getMarshaller()));
-        cfg.deploymentMode(compactObject(c.getDeploymentMode()));
-        cfg.daemon(boolValue(IGNITE_DAEMON, c.isDaemon()));
-        cfg.jmxRemote(ignite.isJmxRemoteEnabled());
-        cfg.restart(ignite.isRestartEnabled());
-        cfg.networkTimeout(c.getNetworkTimeout());
-        cfg.logger(compactClass(c.getGridLogger()));
-        cfg.discoStartupDelay(c.getDiscoveryStartupDelay());
-        cfg.mBeanServer(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.programName(getProperty(IGNITE_PROG_NAME));
-        cfg.quiet(boolValue(IGNITE_QUIET, true));
-        cfg.successFile(getProperty(IGNITE_SUCCESS_FILE));
-        cfg.updateNotifier(boolValue(IGNITE_UPDATE_NOTIFIER, true));
+        cfg.gridName = c.getGridName();
+        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.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;
     }
@@ -136,13 +136,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param gridName New grid name.
-     */
-    public void gridName(@Nullable String gridName) {
-        this.gridName = gridName;
-    }
-
-    /**
      * @return IGNITE_HOME determined at startup.
      */
     @Nullable public String ggHome() {
@@ -150,13 +143,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param ggHome New IGNITE_HOME determined at startup.
-     */
-    public void ggHome(@Nullable String ggHome) {
-        this.ggHome = ggHome;
-    }
-
-    /**
      * @return Local host value used.
      */
     @Nullable public String localHost() {
@@ -164,13 +150,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param locHost New local host value used.
-     */
-    public void localHost(@Nullable String locHost) {
-        this.locHost = locHost;
-    }
-
-    /**
      * @return Node id.
      */
     public UUID nodeId() {
@@ -178,13 +157,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param nodeId New node id.
-     */
-    public void nodeId(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /**
      * @return Marshaller used.
      */
     public String marshaller() {
@@ -192,13 +164,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param marsh New marshaller used.
-     */
-    public void marshaller(String marsh) {
-        this.marsh = marsh;
-    }
-
-    /**
      * @return Deployment Mode.
      */
     public Object deploymentMode() {
@@ -206,13 +171,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param deployMode New Deployment Mode.
-     */
-    public void deploymentMode(Object deployMode) {
-        this.deployMode = deployMode;
-    }
-
-    /**
      * @return Whether this node daemon or not.
      */
     public boolean daemon() {
@@ -220,13 +178,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param daemon New whether this node daemon or not.
-     */
-    public void daemon(boolean daemon) {
-        this.daemon = daemon;
-    }
-
-    /**
      * @return Whether remote JMX is enabled.
      */
     public boolean jmxRemote() {
@@ -234,13 +185,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param jmxRemote New whether remote JMX is enabled.
-     */
-    public void jmxRemote(boolean jmxRemote) {
-        this.jmxRemote = jmxRemote;
-    }
-
-    /**
      * @return Is node restart enabled.
      */
     public boolean restart() {
@@ -248,13 +192,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param restart New is node restart enabled.
-     */
-    public void restart(boolean restart) {
-        this.restart = restart;
-    }
-
-    /**
      * @return Network timeout.
      */
     public long networkTimeout() {
@@ -262,13 +199,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param netTimeout New network timeout.
-     */
-    public void networkTimeout(long netTimeout) {
-        this.netTimeout = netTimeout;
-    }
-
-    /**
      * @return Logger used on node.
      */
     public String logger() {
@@ -276,13 +206,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param log New logger used on node.
-     */
-    public void logger(String log) {
-        this.log = log;
-    }
-
-    /**
      * @return Discovery startup delay.
      */
     public long discoStartupDelay() {
@@ -290,13 +213,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param discoStartupDelay New discovery startup delay.
-     */
-    public void discoStartupDelay(long discoStartupDelay) {
-        this.discoStartupDelay = discoStartupDelay;
-    }
-
-    /**
      * @return MBean server name
      */
     @Nullable public String mBeanServer() {
@@ -304,13 +220,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param mBeanSrv New mBean server name
-     */
-    public void mBeanServer(@Nullable String mBeanSrv) {
-        this.mBeanSrv = mBeanSrv;
-    }
-
-    /**
      * @return Whether ASCII logo is disabled.
      */
     public boolean noAscii() {
@@ -318,13 +227,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param noAscii New whether ASCII logo is disabled.
-     */
-    public void noAscii(boolean noAscii) {
-        this.noAscii = noAscii;
-    }
-
-    /**
      * @return Whether no discovery order is allowed.
      */
     public boolean noDiscoOrder() {
@@ -332,13 +234,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param noDiscoOrder New whether no discovery order is allowed.
-     */
-    public void noDiscoOrder(boolean noDiscoOrder) {
-        this.noDiscoOrder = noDiscoOrder;
-    }
-
-    /**
      * @return Whether shutdown hook is disabled.
      */
     public boolean noShutdownHook() {
@@ -346,13 +241,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param noShutdownHook New whether shutdown hook is disabled.
-     */
-    public void noShutdownHook(boolean noShutdownHook) {
-        this.noShutdownHook = noShutdownHook;
-    }
-
-    /**
      * @return Name of command line program.
      */
     public String programName() {
@@ -360,13 +248,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param progName New name of command line program.
-     */
-    public void programName(String progName) {
-        this.progName = progName;
-    }
-
-    /**
      * @return Whether node is in quiet mode.
      */
     public boolean quiet() {
@@ -374,13 +255,6 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param quiet New whether node is in quiet mode.
-     */
-    public void quiet(boolean quiet) {
-        this.quiet = quiet;
-    }
-
-    /**
      * @return Success file name.
      */
     public String successFile() {
@@ -388,26 +262,12 @@ public class VisorBasicConfiguration implements Serializable {
     }
 
     /**
-     * @param successFile New success file name.
-     */
-    public void successFile(String successFile) {
-        this.successFile = successFile;
-    }
-
-    /**
      * @return Whether update checker is enabled.
      */
     public boolean updateNotifier() {
         return updateNtf;
     }
 
-    /**
-     * @param updateNtf New whether update checker is enabled.
-     */
-    public void updateNotifier(boolean updateNtf) {
-        this.updateNtf = updateNtf;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorBasicConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorCacheQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorCacheQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorCacheQueryConfiguration.java
index 882fa34..8d284c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorCacheQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorCacheQueryConfiguration.java
@@ -58,12 +58,12 @@ public class VisorCacheQueryConfiguration implements Serializable {
             QueryTypeResolver rslvr = qccfg.getTypeResolver();
 
             if (rslvr != null)
-                cfg.typeResolver(U.compact(rslvr.getClass().getName()));
+                cfg.typeRslvr = U.compact(rslvr.getClass().getName());
 
-            cfg.indexPrimitiveKey(qccfg.isIndexPrimitiveKey());
-            cfg.indexPrimitiveValue(qccfg.isIndexPrimitiveValue());
-            cfg.indexFixedTyping(qccfg.isIndexFixedTyping());
-            cfg.escapeAll(qccfg.isEscapeAll());
+            cfg.idxPrimitiveKey = qccfg.isIndexPrimitiveKey();
+            cfg.idxPrimitiveVal = qccfg.isIndexPrimitiveValue();
+            cfg.idxFixedTyping = qccfg.isIndexFixedTyping();
+            cfg.escapeAll = qccfg.isEscapeAll();
         }
 
         return cfg;
@@ -77,13 +77,6 @@ public class VisorCacheQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param typeRslvr Query type resolver class name.
-     */
-    public void typeResolver(String typeRslvr) {
-        this.typeRslvr = typeRslvr;
-    }
-
-    /**
      * @return {@code true} if primitive keys should be indexed.
      */
     public boolean indexPrimitiveKey() {
@@ -91,13 +84,6 @@ public class VisorCacheQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param idxPrimitiveKey {@code true} if primitive keys should be indexed.
-     */
-    public void indexPrimitiveKey(boolean idxPrimitiveKey) {
-        this.idxPrimitiveKey = idxPrimitiveKey;
-    }
-
-    /**
      * @return {@code true} if primitive values should be indexed.
      */
     public boolean indexPrimitiveValue() {
@@ -105,13 +91,6 @@ public class VisorCacheQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param idxPrimitiveVal {@code true} if primitive values should be indexed.
-     */
-    public void indexPrimitiveValue(boolean idxPrimitiveVal) {
-        this.idxPrimitiveVal = idxPrimitiveVal;
-    }
-
-    /**
      * @return {@code true} if SQL engine should try to convert values to their respective SQL types.
      */
     public boolean indexFixedTyping() {
@@ -119,23 +98,9 @@ public class VisorCacheQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param idxFixedTyping {@code true} if SQL engine should try to convert values to their respective SQL types.
-     */
-    public void indexFixedTyping(boolean idxFixedTyping) {
-        this.idxFixedTyping = idxFixedTyping;
-    }
-
-    /**
      * @return {@code true} if SQL engine generate SQL statements with escaped names.
      */
     public boolean escapeAll() {
         return escapeAll;
     }
-
-    /**
-     * @param escapeAll {@code true} if SQL engine should generate SQL statements with escaped names.
-     */
-    public void escapeAll(boolean escapeAll) {
-        this.escapeAll = escapeAll;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 05f407e..8bc891e 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
@@ -54,16 +54,16 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     public static VisorExecutorServiceConfiguration from(IgniteConfiguration c) {
         VisorExecutorServiceConfiguration cfg = new VisorExecutorServiceConfiguration();
 
-        cfg.publicThreadPoolSize(c.getPublicThreadPoolSize());
-        cfg.systemThreadPoolSize(c.getSystemThreadPoolSize());
-        cfg.managementThreadPoolSize(c.getManagementThreadPoolSize());
-        cfg.peerClassLoadingThreadPoolSize(c.getPeerClassLoadingThreadPoolSize());
-        cfg.igfsThreadPoolSize(c.getIgfsThreadPoolSize());
+        cfg.pubPoolSize = c.getPublicThreadPoolSize();
+        cfg.sysPoolSz = c.getSystemThreadPoolSize();
+        cfg.mgmtPoolSize = c.getManagementThreadPoolSize();
+        cfg.p2pPoolSz = c.getPeerClassLoadingThreadPoolSize();
+        cfg.igfsPoolSize = c.getIgfsThreadPoolSize();
 
         ConnectorConfiguration cc = c.getConnectorConfiguration();
 
         if (cc != null)
-            cfg.restThreadPoolSize(cc.getThreadPoolSize());
+            cfg.restPoolSz = cc.getThreadPoolSize();
 
         return cfg;
     }
@@ -76,13 +76,6 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     }
 
     /**
-     * @param pubPoolSize Public pool size.
-     */
-    public void publicThreadPoolSize(int pubPoolSize) {
-        this.pubPoolSize = pubPoolSize;
-    }
-
-    /**
      * @return System pool size.
      */
     public int systemThreadPoolSize() {
@@ -90,13 +83,6 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     }
 
     /**
-     * @param sysPoolSz System pool size.
-     */
-    public void systemThreadPoolSize(int sysPoolSz) {
-        this.sysPoolSz = sysPoolSz;
-    }
-
-    /**
      * @return Management pool size.
      */
     public int managementThreadPoolSize() {
@@ -104,13 +90,6 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     }
 
     /**
-     * @param mgmtPoolSize New Management pool size.
-     */
-    public void managementThreadPoolSize(int mgmtPoolSize) {
-        this.mgmtPoolSize = mgmtPoolSize;
-    }
-
-    /**
      * @return IGFS pool size.
      */
     public int igfsThreadPoolSize() {
@@ -118,13 +97,6 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     }
 
     /**
-     * @param igfsPoolSize New iGFS pool size.
-     */
-    public void igfsThreadPoolSize(int igfsPoolSize) {
-        this.igfsPoolSize = igfsPoolSize;
-    }
-
-    /**
      * @return Peer-to-peer pool size.
      */
     public int peerClassLoadingThreadPoolSize() {
@@ -132,26 +104,12 @@ public class VisorExecutorServiceConfiguration implements Serializable {
     }
 
     /**
-     * @param p2pPoolSz New peer-to-peer pool size.
-     */
-    public void peerClassLoadingThreadPoolSize(int p2pPoolSz) {
-        this.p2pPoolSz = p2pPoolSz;
-    }
-
-    /**
      * @return REST requests pool size.
      */
     public int restThreadPoolSize() {
         return restPoolSz;
     }
 
-    /**
-     * @param restPoolSz REST requests pool size.
-     */
-    public void restThreadPoolSize(int restPoolSz) {
-        this.restPoolSz = restPoolSz;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorExecutorServiceConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 446db32..96c69d9 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
@@ -102,25 +102,25 @@ public class VisorGridConfiguration implements Serializable {
 
         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));
-        executeService(VisorExecutorServiceConfiguration.from(c));
-        segmentation(VisorSegmentationConfiguration.from(c));
-        includeProperties(compactArray(c.getIncludeProperties()));
-        includeEventTypes(c.getIncludeEventTypes());
-        rest(VisorRestConfiguration.from(c));
-        userAttributes(c.getUserAttributes());
-        caches(VisorCacheConfiguration.list(ignite, c.getCacheConfiguration()));
-        igfss(VisorIgfsConfiguration.list(c.getIgfsConfiguration()));
-        streamers(VisorStreamerConfiguration.list(c.getStreamerConfiguration()));
-        env(new HashMap<>(getenv()));
-        systemProperties(getProperties());
-        atomic(VisorAtomicConfiguration.from(c.getAtomicConfiguration()));
-        transaction(VisorTransactionConfiguration.from(c.getTransactionConfiguration()));
-        queryConfiguration(VisorQueryConfiguration.from(c.getQueryConfiguration()));
+        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);
+        inclProps = compactArray(c.getIncludeProperties());
+        inclEvtTypes = c.getIncludeEventTypes();
+        rest = VisorRestConfiguration.from(c);
+        userAttrs = c.getUserAttributes();
+        caches = VisorCacheConfiguration.list(ignite, c.getCacheConfiguration());
+        igfss = VisorIgfsConfiguration.list(c.getIgfsConfiguration());
+        streamers = VisorStreamerConfiguration.list(c.getStreamerConfiguration());
+        env = new HashMap<>(getenv());
+        sysProps = getProperties();
+        atomic = VisorAtomicConfiguration.from(c.getAtomicConfiguration());
+        txCfg = VisorTransactionConfiguration.from(c.getTransactionConfiguration());
+        qryCfg = VisorQueryConfiguration.from(c.getQueryConfiguration());
 
         return this;
     }
@@ -133,13 +133,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param basic New basic.
-     */
-    public void basic(VisorBasicConfiguration basic) {
-        this.basic = basic;
-    }
-
-    /**
      * @return Metrics.
      */
     public VisorMetricsConfiguration metrics() {
@@ -147,13 +140,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param metrics New metrics.
-     */
-    public void metrics(VisorMetricsConfiguration metrics) {
-        this.metrics = metrics;
-    }
-
-    /**
      * @return SPIs.
      */
     public VisorSpisConfiguration spis() {
@@ -161,13 +147,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param spis New SPIs.
-     */
-    public void spis(VisorSpisConfiguration spis) {
-        this.spis = spis;
-    }
-
-    /**
      * @return P2P.
      */
     public VisorPeerToPeerConfiguration p2p() {
@@ -175,13 +154,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param p2P New p2p.
-     */
-    public void p2p(VisorPeerToPeerConfiguration p2P) {
-        p2p = p2P;
-    }
-
-    /**
      * @return Lifecycle.
      */
     public VisorLifecycleConfiguration lifecycle() {
@@ -189,13 +161,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param lifecycle New lifecycle.
-     */
-    public void lifecycle(VisorLifecycleConfiguration lifecycle) {
-        this.lifecycle = lifecycle;
-    }
-
-    /**
      * @return Executors service configuration.
      */
     public VisorExecutorServiceConfiguration executeService() {
@@ -203,13 +168,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param execSvc New executors service configuration.
-     */
-    public void executeService(VisorExecutorServiceConfiguration execSvc) {
-        this.execSvc = execSvc;
-    }
-
-    /**
      * @return Segmentation.
      */
     public VisorSegmentationConfiguration segmentation() {
@@ -217,13 +175,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param seg New segmentation.
-     */
-    public void segmentation(VisorSegmentationConfiguration seg) {
-        this.seg = seg;
-    }
-
-    /**
      * @return Include properties.
      */
     public String includeProperties() {
@@ -231,13 +182,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param inclProps New include properties.
-     */
-    public void includeProperties(String inclProps) {
-        this.inclProps = inclProps;
-    }
-
-    /**
      * @return Include events types.
      */
     public int[] includeEventTypes() {
@@ -245,13 +189,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param inclEvtTypes New include events types.
-     */
-    public void includeEventTypes(int[] inclEvtTypes) {
-        this.inclEvtTypes = inclEvtTypes;
-    }
-
-    /**
      * @return Rest.
      */
     public VisorRestConfiguration rest() {
@@ -259,13 +196,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param rest New rest.
-     */
-    public void rest(VisorRestConfiguration rest) {
-        this.rest = rest;
-    }
-
-    /**
      * @return User attributes.
      */
     public Map<String, ?> userAttributes() {
@@ -273,13 +203,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param userAttrs New user attributes.
-     */
-    public void userAttributes(Map<String, ?> userAttrs) {
-        this.userAttrs = userAttrs;
-    }
-
-    /**
      * @return Caches.
      */
     public Iterable<VisorCacheConfiguration> caches() {
@@ -287,13 +210,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param caches New caches.
-     */
-    public void caches(Iterable<VisorCacheConfiguration> caches) {
-        this.caches = caches;
-    }
-
-    /**
      * @return Igfss.
      */
     public Iterable<VisorIgfsConfiguration> igfss() {
@@ -301,13 +217,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param igfss New igfss.
-     */
-    public void igfss(Iterable<VisorIgfsConfiguration> igfss) {
-        this.igfss = igfss;
-    }
-
-    /**
      * @return Streamers.
      */
     public Iterable<VisorStreamerConfiguration> streamers() {
@@ -315,13 +224,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param streamers New streamers.
-     */
-    public void streamers(Iterable<VisorStreamerConfiguration> streamers) {
-        this.streamers = streamers;
-    }
-
-    /**
      * @return Environment.
      */
     public Map<String, String> env() {
@@ -329,13 +231,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param env New environment.
-     */
-    public void env(Map<String, String> env) {
-        this.env = env;
-    }
-
-    /**
      * @return System properties.
      */
     public Properties systemProperties() {
@@ -343,13 +238,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param sysProps New system properties.
-     */
-    public void systemProperties(Properties sysProps) {
-        this.sysProps = sysProps;
-    }
-
-    /**
      * @return Configuration of atomic data structures.
      */
     public VisorAtomicConfiguration atomic() {
@@ -357,13 +245,6 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param atomic New configuration of atomic data structures.
-     */
-    public void atomic(VisorAtomicConfiguration atomic) {
-        this.atomic = atomic;
-    }
-
-    /**
      * @return Transactions configuration.
      */
     public VisorTransactionConfiguration transaction() {
@@ -371,28 +252,14 @@ public class VisorGridConfiguration implements Serializable {
     }
 
     /**
-     * @param txCfg New transactions configuration.
-     */
-    public void transaction(VisorTransactionConfiguration txCfg) {
-        this.txCfg = txCfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(VisorGridConfiguration.class, this);
-    }
-
-    /**
      * @return Query configuration.
      */
     public VisorQueryConfiguration queryConfiguration() {
         return qryCfg;
     }
 
-    /**
-     * @param qryCfg New query configuration.
-     */
-    public void queryConfiguration(VisorQueryConfiguration qryCfg) {
-        this.qryCfg = qryCfg;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorGridConfiguration.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 287de27..056ac7f 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
@@ -128,44 +128,44 @@ public class VisorIgfsConfiguration implements Serializable {
     public static VisorIgfsConfiguration from(IgfsConfiguration igfs) {
         VisorIgfsConfiguration cfg = new VisorIgfsConfiguration();
 
-        cfg.name(igfs.getName());
-        cfg.metaCacheName(igfs.getMetaCacheName());
-        cfg.dataCacheName(igfs.getDataCacheName());
-        cfg.blockSize(igfs.getBlockSize());
-        cfg.prefetchBlocks(igfs.getPrefetchBlocks());
-        cfg.streamBufferSize(igfs.getStreamBufferSize());
-        cfg.perNodeBatchSize(igfs.getPerNodeBatchSize());
-        cfg.perNodeParallelBatchCount(igfs.getPerNodeParallelBatchCount());
+        cfg.name = igfs.getName();
+        cfg.metaCacheName = igfs.getMetaCacheName();
+        cfg.dataCacheName = igfs.getDataCacheName();
+        cfg.blockSize = igfs.getBlockSize();
+        cfg.prefetchBlocks = igfs.getPrefetchBlocks();
+        cfg.streamBufSize = igfs.getStreamBufferSize();
+        cfg.perNodeBatchSize = igfs.getPerNodeBatchSize();
+        cfg.perNodeParallelBatchCnt = igfs.getPerNodeParallelBatchCount();
 
         Igfs secFs = igfs.getSecondaryFileSystem();
 
         if (secFs != null) {
             Map<String, String> props = secFs.properties();
 
-            cfg.secondaryHadoopFileSystemUri(props.get(SECONDARY_FS_URI));
-            cfg.secondaryHadoopFileSystemConfigPath(props.get(SECONDARY_FS_CONFIG_PATH));
+            cfg.secondaryHadoopFileSysUri = props.get(SECONDARY_FS_URI);
+            cfg.secondaryHadoopFileSysCfgPath = props.get(SECONDARY_FS_CONFIG_PATH);
         }
 
-        cfg.defaultMode(igfs.getDefaultMode());
-        cfg.pathModes(igfs.getPathModes());
-        cfg.dualModePutExecutorService(compactClass(igfs.getDualModePutExecutorService()));
-        cfg.dualModePutExecutorServiceShutdown(igfs.getDualModePutExecutorServiceShutdown());
-        cfg.dualModeMaxPendingPutsSize(igfs.getDualModeMaxPendingPutsSize());
-        cfg.maxTaskRangeLength(igfs.getMaximumTaskRangeLength());
-        cfg.fragmentizerConcurrentFiles(igfs.getFragmentizerConcurrentFiles());
-        cfg.fragmentizerLocalWritesRatio(igfs.getFragmentizerLocalWritesRatio());
-        cfg.fragmentizerEnabled(igfs.isFragmentizerEnabled());
-        cfg.fragmentizerThrottlingBlockLength(igfs.getFragmentizerThrottlingBlockLength());
-        cfg.fragmentizerThrottlingDelay(igfs.getFragmentizerThrottlingDelay());
+        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();
 
         Map<String, String> endpointCfg = igfs.getIpcEndpointConfiguration();
-        cfg.ipcEndpointConfiguration(endpointCfg != null ? endpointCfg.toString() : null);
+        cfg.ipcEndpointCfg = endpointCfg != null ? endpointCfg.toString() : null;
 
-        cfg.ipcEndpointEnabled(igfs.isIpcEndpointEnabled());
-        cfg.maxSpace(igfs.getMaxSpaceSize());
-        cfg.managementPort(igfs.getManagementPort());
-        cfg.sequenceReadsBeforePrefetch(igfs.getSequentialReadsBeforePrefetch());
-        cfg.trashPurgeTimeout(igfs.getTrashPurgeTimeout());
+        cfg.ipcEndpointEnabled = igfs.isIpcEndpointEnabled();
+        cfg.maxSpace = igfs.getMaxSpaceSize();
+        cfg.mgmtPort = igfs.getManagementPort();
+        cfg.seqReadsBeforePrefetch = igfs.getSequentialReadsBeforePrefetch();
+        cfg.trashPurgeTimeout = igfs.getTrashPurgeTimeout();
 
         return cfg;
     }
@@ -196,13 +196,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param name New IGFS instance name.
-     */
-    public void name(@Nullable String name) {
-        this.name = name;
-    }
-
-    /**
      * @return Cache name to store IGFS meta information.
      */
     @Nullable public String metaCacheName() {
@@ -210,13 +203,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param metaCacheName New cache name to store IGFS meta information.
-     */
-    public void metaCacheName(@Nullable String metaCacheName) {
-        this.metaCacheName = metaCacheName;
-    }
-
-    /**
      * @return Cache name to store IGFS data.
      */
     @Nullable public String dataCacheName() {
@@ -224,13 +210,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param dataCacheName New cache name to store IGFS data.
-     */
-    public void dataCacheName(@Nullable String dataCacheName) {
-        this.dataCacheName = dataCacheName;
-    }
-
-    /**
      * @return File's data block size.
      */
     public int blockSize() {
@@ -238,13 +217,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param blockSize New file's data block size.
-     */
-    public void blockSize(int blockSize) {
-        this.blockSize = blockSize;
-    }
-
-    /**
      * @return Number of pre-fetched blocks if specific file's chunk is requested.
      */
     public int prefetchBlocks() {
@@ -252,13 +224,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param prefetchBlocks New number of pre-fetched blocks if specific file's chunk is requested.
-     */
-    public void prefetchBlocks(int prefetchBlocks) {
-        this.prefetchBlocks = prefetchBlocks;
-    }
-
-    /**
      * @return Read/write buffer size for IGFS stream operations in bytes.
      */
     public int streamBufferSize() {
@@ -266,13 +231,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param streamBufSize New read/write buffer size for IGFS stream operations in bytes.
-     */
-    public void streamBufferSize(int streamBufSize) {
-        this.streamBufSize = streamBufSize;
-    }
-
-    /**
      * @return Number of file blocks buffered on local node before sending batch to remote node.
      */
     public int perNodeBatchSize() {
@@ -280,13 +238,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param perNodeBatchSize New number of file blocks buffered on local node before sending batch to remote node.
-     */
-    public void perNodeBatchSize(int perNodeBatchSize) {
-        this.perNodeBatchSize = perNodeBatchSize;
-    }
-
-    /**
      * @return Number of batches that can be concurrently sent to remote node.
      */
     public int perNodeParallelBatchCount() {
@@ -294,13 +245,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param perNodeParallelBatchCnt New number of batches that can be concurrently sent to remote node.
-     */
-    public void perNodeParallelBatchCount(int perNodeParallelBatchCnt) {
-        this.perNodeParallelBatchCnt = perNodeParallelBatchCnt;
-    }
-
-    /**
      * @return URI of the secondary Hadoop file system.
      */
     @Nullable public String secondaryHadoopFileSystemUri() {
@@ -308,13 +252,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param secondaryHadoopFileSysUri New URI of the secondary Hadoop file system.
-     */
-    public void secondaryHadoopFileSystemUri(@Nullable String secondaryHadoopFileSysUri) {
-        this.secondaryHadoopFileSysUri = secondaryHadoopFileSysUri;
-    }
-
-    /**
      * @return Path for the secondary hadoop file system config.
      */
     @Nullable public String secondaryHadoopFileSystemConfigPath() {
@@ -322,13 +259,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param secondaryHadoopFileSysCfgPath New path for the secondary hadoop file system config.
-     */
-    public void secondaryHadoopFileSystemConfigPath(@Nullable String secondaryHadoopFileSysCfgPath) {
-        this.secondaryHadoopFileSysCfgPath = secondaryHadoopFileSysCfgPath;
-    }
-
-    /**
      * @return IGFS instance mode.
      */
     public IgfsMode defaultMode() {
@@ -336,13 +266,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param dfltMode New IGFS instance mode.
-     */
-    public void defaultMode(IgfsMode dfltMode) {
-        this.dfltMode = dfltMode;
-    }
-
-    /**
      * @return Map of paths to IGFS modes.
      */
     @Nullable public Map<String, IgfsMode> pathModes() {
@@ -350,13 +273,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param pathModes New map of paths to IGFS modes.
-     */
-    public void pathModes(@Nullable Map<String, IgfsMode> pathModes) {
-        this.pathModes = pathModes;
-    }
-
-    /**
      * @return Dual mode PUT operations executor service.
      */
     public String dualModePutExecutorService() {
@@ -364,13 +280,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param dualModePutExecutorSrvc New dual mode PUT operations executor service.
-     */
-    public void dualModePutExecutorService(String dualModePutExecutorSrvc) {
-        this.dualModePutExecutorSrvc = dualModePutExecutorSrvc;
-    }
-
-    /**
      * @return Dual mode PUT operations executor service shutdown flag.
      */
     public boolean dualModePutExecutorServiceShutdown() {
@@ -378,13 +287,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param dualModePutExecutorSrvcShutdown New dual mode PUT operations executor service shutdown flag.
-     */
-    public void dualModePutExecutorServiceShutdown(boolean dualModePutExecutorSrvcShutdown) {
-        this.dualModePutExecutorSrvcShutdown = dualModePutExecutorSrvcShutdown;
-    }
-
-    /**
      * @return Maximum amount of data in pending puts.
      */
     public long dualModeMaxPendingPutsSize() {
@@ -392,13 +294,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param dualModeMaxPendingPutsSize New maximum amount of data in pending puts.
-     */
-    public void dualModeMaxPendingPutsSize(long dualModeMaxPendingPutsSize) {
-        this.dualModeMaxPendingPutsSize = dualModeMaxPendingPutsSize;
-    }
-
-    /**
      * @return Maximum range length.
      */
     public long maxTaskRangeLength() {
@@ -406,13 +301,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param maxTaskRangeLen New maximum range length.
-     */
-    public void maxTaskRangeLength(long maxTaskRangeLen) {
-        this.maxTaskRangeLen = maxTaskRangeLen;
-    }
-
-    /**
      * @return Fragmentizer concurrent files.
      */
     public int fragmentizerConcurrentFiles() {
@@ -420,13 +308,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param fragmentizerConcurrentFiles New fragmentizer concurrent files.
-     */
-    public void fragmentizerConcurrentFiles(int fragmentizerConcurrentFiles) {
-        this.fragmentizerConcurrentFiles = fragmentizerConcurrentFiles;
-    }
-
-    /**
      * @return Fragmentizer local writes ratio.
      */
     public float fragmentizerLocalWritesRatio() {
@@ -434,13 +315,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param fragmentizerLocWritesRatio New fragmentizer local writes ratio.
-     */
-    public void fragmentizerLocalWritesRatio(float fragmentizerLocWritesRatio) {
-        this.fragmentizerLocWritesRatio = fragmentizerLocWritesRatio;
-    }
-
-    /**
      * @return Fragmentizer enabled flag.
      */
     public boolean fragmentizerEnabled() {
@@ -448,13 +322,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param fragmentizerEnabled New fragmentizer enabled flag.
-     */
-    public void fragmentizerEnabled(boolean fragmentizerEnabled) {
-        this.fragmentizerEnabled = fragmentizerEnabled;
-    }
-
-    /**
      * @return Fragmentizer throttling block length.
      */
     public long fragmentizerThrottlingBlockLength() {
@@ -462,13 +329,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param fragmentizerThrottlingBlockLen New fragmentizer throttling block length.
-     */
-    public void fragmentizerThrottlingBlockLength(long fragmentizerThrottlingBlockLen) {
-        this.fragmentizerThrottlingBlockLen = fragmentizerThrottlingBlockLen;
-    }
-
-    /**
      * @return Fragmentizer throttling delay.
      */
     public long fragmentizerThrottlingDelay() {
@@ -476,13 +336,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param fragmentizerThrottlingDelay New fragmentizer throttling delay.
-     */
-    public void fragmentizerThrottlingDelay(long fragmentizerThrottlingDelay) {
-        this.fragmentizerThrottlingDelay = fragmentizerThrottlingDelay;
-    }
-
-    /**
      * @return IPC endpoint config (in JSON format) to publish IGFS over.
      */
     @Nullable public String ipcEndpointConfiguration() {
@@ -490,13 +343,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param ipcEndpointCfg New IPC endpoint config (in JSON format) to publish IGFS over.
-     */
-    public void ipcEndpointConfiguration(@Nullable String ipcEndpointCfg) {
-        this.ipcEndpointCfg = ipcEndpointCfg;
-    }
-
-    /**
      * @return IPC endpoint enabled flag.
      */
     public boolean ipcEndpointEnabled() {
@@ -504,13 +350,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param ipcEndpointEnabled New iPC endpoint enabled flag.
-     */
-    public void ipcEndpointEnabled(boolean ipcEndpointEnabled) {
-        this.ipcEndpointEnabled = ipcEndpointEnabled;
-    }
-
-    /**
      * @return Maximum space.
      */
     public long maxSpace() {
@@ -518,13 +357,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param maxSpace New maximum space.
-     */
-    public void maxSpace(long maxSpace) {
-        this.maxSpace = maxSpace;
-    }
-
-    /**
      * @return Management port.
      */
     public int managementPort() {
@@ -532,13 +364,6 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param mgmtPort New management port.
-     */
-    public void managementPort(int mgmtPort) {
-        this.mgmtPort = mgmtPort;
-    }
-
-    /**
      * @return Amount of sequential block reads before prefetch is triggered.
      */
     public int sequenceReadsBeforePrefetch() {
@@ -546,29 +371,14 @@ public class VisorIgfsConfiguration implements Serializable {
     }
 
     /**
-     * @param seqReadsBeforePrefetch New amount of sequential block reads before prefetch is triggered.
-     */
-    public void sequenceReadsBeforePrefetch(int seqReadsBeforePrefetch) {
-        this.seqReadsBeforePrefetch = seqReadsBeforePrefetch;
-    }
-
-    /**
      * @return Trash purge await timeout.
      */
     public long trashPurgeTimeout() {
         return trashPurgeTimeout;
     }
 
-    /**
-     * @param trashPurgeTimeout New trash purge await timeout.
-     */
-    public void trashPurgeTimeout(long trashPurgeTimeout) {
-        this.trashPurgeTimeout = trashPurgeTimeout;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorIgfsConfiguration.class, this);
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 723fbee..3e0f9cc 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
@@ -42,7 +42,7 @@ public class VisorLifecycleConfiguration implements Serializable {
     public static VisorLifecycleConfiguration from(IgniteConfiguration c) {
         VisorLifecycleConfiguration cfg = new VisorLifecycleConfiguration();
 
-        cfg.beans(compactArray(c.getLifecycleBeans()));
+        cfg.beans = compactArray(c.getLifecycleBeans());
 
         return cfg;
     }
@@ -54,13 +54,6 @@ public class VisorLifecycleConfiguration implements Serializable {
         return beans;
     }
 
-    /**
-     * @param beans New lifecycle beans.
-     */
-    public void beans(@Nullable String beans) {
-        this.beans = beans;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorLifecycleConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/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 51dc092..e76adac 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.visor.node;
 
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.io.*;
 
@@ -44,9 +45,9 @@ public class VisorMetricsConfiguration implements Serializable {
     public static VisorMetricsConfiguration from(IgniteConfiguration c) {
         VisorMetricsConfiguration cfg = new VisorMetricsConfiguration();
 
-        cfg.expireTime(c.getMetricsExpireTime());
-        cfg.historySize(c.getMetricsHistorySize());
-        cfg.loggerFrequency(c.getMetricsLogFrequency());
+        cfg.expTime = c.getMetricsExpireTime();
+        cfg.histSize = c.getMetricsHistorySize();
+        cfg.logFreq = c.getMetricsLogFrequency();
 
         return cfg;
     }
@@ -59,13 +60,6 @@ public class VisorMetricsConfiguration implements Serializable {
     }
 
     /**
-     * @param expTime New metrics expire time.
-     */
-    public void expireTime(long expTime) {
-        this.expTime = expTime;
-    }
-
-    /**
      * @return Number of node metrics stored in memory.
      */
     public int historySize() {
@@ -73,23 +67,14 @@ public class VisorMetricsConfiguration implements Serializable {
     }
 
     /**
-     * @param histSize New number of node metrics stored in memory.
-     */
-    public void historySize(int histSize) {
-        this.histSize = histSize;
-    }
-
-    /**
      * @return Frequency of metrics log printout.
      */
     public long loggerFrequency() {
         return logFreq;
     }
 
-    /**
-     * @param logFreq New frequency of metrics log printout.
-     */
-    public void loggerFrequency(long logFreq) {
-        this.logFreq = logFreq;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorMetricsConfiguration.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPeerToPeerConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPeerToPeerConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPeerToPeerConfiguration.java
index 7f0d050..5c4f8fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPeerToPeerConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorPeerToPeerConfiguration.java
@@ -48,9 +48,9 @@ public class VisorPeerToPeerConfiguration implements Serializable {
     public static VisorPeerToPeerConfiguration from(IgniteConfiguration c) {
         VisorPeerToPeerConfiguration cfg = new VisorPeerToPeerConfiguration();
 
-        cfg.p2pEnabled(c.isPeerClassLoadingEnabled());
-        cfg.p2pMissedResponseCacheSize(c.getPeerClassLoadingMissedResourcesCacheSize());
-        cfg.p2pLocalClassPathExclude(compactArray(c.getPeerClassLoadingLocalClassPathExclude()));
+        cfg.p2pEnabled = c.isPeerClassLoadingEnabled();
+        cfg.p2pMissedResCacheSize = c.getPeerClassLoadingMissedResourcesCacheSize();
+        cfg.p2pLocClsPathExcl = compactArray(c.getPeerClassLoadingLocalClassPathExclude());
 
         return cfg;
     }
@@ -63,13 +63,6 @@ public class VisorPeerToPeerConfiguration implements Serializable {
     }
 
     /**
-     * @param p2pEnabled New whether peer-to-peer class loading is enabled.
-     */
-    public void p2pEnabled(boolean p2pEnabled) {
-        this.p2pEnabled = p2pEnabled;
-    }
-
-    /**
      * @return Missed resource cache size.
      */
     public int p2pMissedResponseCacheSize() {
@@ -77,27 +70,12 @@ public class VisorPeerToPeerConfiguration implements Serializable {
     }
 
     /**
-     * @param p2pMissedResCacheSize New missed resource cache size.
-     */
-    public void p2pMissedResponseCacheSize(int p2pMissedResCacheSize) {
-        this.p2pMissedResCacheSize = p2pMissedResCacheSize;
-    }
-
-    /**
      * @return List of packages from the system classpath that need to be loaded from task originating node.
      */
     @Nullable public String p2pLocalClassPathExclude() {
         return p2pLocClsPathExcl;
     }
 
-    /**
-     * @param p2pLocClsPathExcl New list of packages from the system classpath that need to be loaded from task
-     * originating node.
-     */
-    public void p2pLocalClassPathExclude(@Nullable String p2pLocClsPathExcl) {
-        this.p2pLocClsPathExcl = p2pLocClsPathExcl;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorPeerToPeerConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorQueryConfiguration.java
index a2599ab..de5e0b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorQueryConfiguration.java
@@ -70,13 +70,13 @@ public class VisorQueryConfiguration implements Serializable {
             for (int i = 0; i < sz; i++)
                 strClss[i] = U.compact(clss[i].getName());
 
-            c.indexCustomFunctionClasses(strClss);
-            c.searchPath(qcfg.getSearchPath());
-            c.initialScriptPath(qcfg.getInitialScriptPath());
-            c.maxOffHeapMemory(qcfg.getMaxOffHeapMemory());
-            c.longQueryExecutionTimeout(qcfg.getLongQueryExecutionTimeout());
-            c.longQueryExplain(qcfg.isLongQueryExplain());
-            c.useOptimizedSerializer(qcfg.isUseOptimizedSerializer());
+            c.idxCustomFuncClss = strClss;
+            c.searchPath = qcfg.getSearchPath();
+            c.initScriptPath = qcfg.getInitialScriptPath();
+            c.maxOffHeapMemory = qcfg.getMaxOffHeapMemory();
+            c.longQryExecTimeout = qcfg.getLongQueryExecutionTimeout();
+            c.longQryExplain = qcfg.isLongQueryExplain();
+            c.useOptimizedSerializer = qcfg.isUseOptimizedSerializer();
         }
 
         return c;
@@ -90,13 +90,6 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param idxCustomFuncClss Classes with methods annotated by {@link QuerySqlFunction}.
-     */
-    public void indexCustomFunctionClasses(String[] idxCustomFuncClss) {
-        this.idxCustomFuncClss = idxCustomFuncClss;
-    }
-
-    /**
      * @return Optional search path consisting of space names to search SQL schema objects.
      */
     public String[] searchPath() {
@@ -104,13 +97,6 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param searchPath Optional search path consisting of space names to search SQL schema objects.
-     */
-    public void searchPath(String[] searchPath) {
-        this.searchPath = searchPath;
-    }
-
-    /**
      * @return Script path to be ran against H2 database after opening.
      */
     public String initialScriptPath() {
@@ -118,13 +104,6 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param initScriptPath Script path to be ran against H2 database after opening.
-     */
-    public void initialScriptPath(String initScriptPath) {
-        this.initScriptPath = initScriptPath;
-    }
-
-    /**
      * @return Maximum amount of memory available to off-heap storage.
      */
     public long maxOffHeapMemory() {
@@ -132,13 +111,6 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param maxOffHeapMemory Maximum amount of memory available to off-heap storage.
-     */
-    public void maxOffHeapMemory(long maxOffHeapMemory) {
-        this.maxOffHeapMemory = maxOffHeapMemory;
-    }
-
-    /**
      * @return Query execution time threshold.
      */
     public long longQueryExecutionTimeout() {
@@ -146,13 +118,6 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param longQryExecTimeout Query execution time threshold.
-     */
-    public void longQueryExecutionTimeout(long longQryExecTimeout) {
-        this.longQryExecTimeout = longQryExecTimeout;
-    }
-
-    /**
      * @return If {@code true}, SPI will print SQL execution plan for long queries.
      */
     public boolean longQryExplain() {
@@ -160,24 +125,14 @@ public class VisorQueryConfiguration implements Serializable {
     }
 
     /**
-     * @param longQryExplain If {@code true}, SPI will print SQL execution plan for long queries.
-     */
-    public void longQueryExplain(boolean longQryExplain) {
-        this.longQryExplain = longQryExplain;
-    }
-
-    /**
      * @return The flag indicating that serializer for H2 database will be set to Ignite's marshaller.
      */
     public boolean useOptimizedSerializer() {
         return useOptimizedSerializer;
     }
 
-    /**
-     * @param useOptimizedSerializer The flag indicating that serializer for H2 database will be set to Ignite's
-     * marshaller.
-     */
-    public void useOptimizedSerializer(boolean useOptimizedSerializer) {
-        this.useOptimizedSerializer = useOptimizedSerializer;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorQueryConfiguration.class, this);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorRestConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorRestConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorRestConfiguration.java
index 1bdb7b7..f279253 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorRestConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorRestConfiguration.java
@@ -72,16 +72,16 @@ public class VisorRestConfiguration implements Serializable {
 
         boolean restEnabled = clnCfg != null;
 
-        cfg.restEnabled(restEnabled);
+        cfg.restEnabled = restEnabled;
 
         if (restEnabled) {
-            cfg.tcpSslEnabled(clnCfg.isSslEnabled());
-            cfg.jettyPath(clnCfg.getJettyPath());
-            cfg.jettyHost(getProperty(IGNITE_JETTY_HOST));
-            cfg.jettyPort(intValue(IGNITE_JETTY_PORT, null));
-            cfg.tcpHost(clnCfg.getHost());
-            cfg.tcpPort(clnCfg.getPort());
-            cfg.tcpSslContextFactory(compactClass(clnCfg.getSslContextFactory()));
+            cfg.tcpSslEnabled = clnCfg.isSslEnabled();
+            cfg.jettyPath = clnCfg.getJettyPath();
+            cfg.jettyHost = getProperty(IGNITE_JETTY_HOST);
+            cfg.jettyPort = intValue(IGNITE_JETTY_PORT, null);
+            cfg.tcpHost = clnCfg.getHost();
+            cfg.tcpPort = clnCfg.getPort();
+            cfg.tcpSslCtxFactory = compactClass(clnCfg.getSslContextFactory());
         }
 
         return cfg;
@@ -93,14 +93,6 @@ public class VisorRestConfiguration implements Serializable {
     public boolean restEnabled() {
         return restEnabled;
     }
-
-    /**
-     * @param restEnabled New whether REST enabled or not.
-     */
-    public void restEnabled(boolean restEnabled) {
-        this.restEnabled = restEnabled;
-    }
-
     /**
      * @return Whether or not SSL is enabled for TCP binary protocol.
      */
@@ -109,13 +101,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param tcpSslEnabled New whether or not SSL is enabled for TCP binary protocol.
-     */
-    public void tcpSslEnabled(boolean tcpSslEnabled) {
-        this.tcpSslEnabled = tcpSslEnabled;
-    }
-
-    /**
      * @return Rest accessible folders (log command can get files from).
      */
     @Nullable public String[] accessibleFolders() {
@@ -123,13 +108,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param accessibleFolders New rest accessible folders (log command can get files from).
-     */
-    public void accessibleFolders(String[] accessibleFolders) {
-        this.accessibleFolders = accessibleFolders;
-    }
-
-    /**
      * @return Jetty config path.
      */
     @Nullable public String jettyPath() {
@@ -137,13 +115,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param jettyPath New jetty config path.
-     */
-    public void jettyPath(String jettyPath) {
-        this.jettyPath = jettyPath;
-    }
-
-    /**
      * @return Jetty host.
      */
     @Nullable public String jettyHost() {
@@ -151,13 +122,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param jettyHost New jetty host.
-     */
-    public void jettyHost(String jettyHost) {
-        this.jettyHost = jettyHost;
-    }
-
-    /**
      * @return Jetty port.
      */
     @Nullable public Integer jettyPort() {
@@ -165,13 +129,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param jettyPort New jetty port.
-     */
-    public void jettyPort(Integer jettyPort) {
-        this.jettyPort = jettyPort;
-    }
-
-    /**
      * @return REST TCP binary host.
      */
     @Nullable public String tcpHost() {
@@ -179,13 +136,6 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param tcpHost New rEST TCP binary host.
-     */
-    public void tcpHost(String tcpHost) {
-        this.tcpHost = tcpHost;
-    }
-
-    /**
      * @return REST TCP binary port.
      */
     @Nullable public Integer tcpPort() {
@@ -193,26 +143,12 @@ public class VisorRestConfiguration implements Serializable {
     }
 
     /**
-     * @param tcpPort New rEST TCP binary port.
-     */
-    public void tcpPort(Integer tcpPort) {
-        this.tcpPort = tcpPort;
-    }
-
-    /**
      * @return Context factory for SSL.
      */
     @Nullable public String tcpSslContextFactory() {
         return tcpSslCtxFactory;
     }
 
-    /**
-     * @param tcpSslCtxFactory New context factory for SSL.
-     */
-    public void tcpSslContextFactory(String tcpSslCtxFactory) {
-        this.tcpSslCtxFactory = tcpSslCtxFactory;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorRestConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSegmentationConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSegmentationConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSegmentationConfiguration.java
index 547e1c6..d712774 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSegmentationConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSegmentationConfiguration.java
@@ -55,11 +55,11 @@ public class VisorSegmentationConfiguration implements Serializable {
     public static VisorSegmentationConfiguration from(IgniteConfiguration c) {
         VisorSegmentationConfiguration cfg = new VisorSegmentationConfiguration();
 
-        cfg.policy(c.getSegmentationPolicy());
-        cfg.resolvers(compactArray(c.getSegmentationResolvers()));
-        cfg.checkFrequency(c.getSegmentCheckFrequency());
-        cfg.waitOnStart(c.isWaitForSegmentOnStart());
-        cfg.passRequired(c.isAllSegmentationResolversPassRequired());
+        cfg.plc = c.getSegmentationPolicy();
+        cfg.resolvers = compactArray(c.getSegmentationResolvers());
+        cfg.checkFreq = c.getSegmentCheckFrequency();
+        cfg.waitOnStart = c.isWaitForSegmentOnStart();
+        cfg.passRequired = c.isAllSegmentationResolversPassRequired();
 
         return cfg;
     }
@@ -72,13 +72,6 @@ public class VisorSegmentationConfiguration implements Serializable {
     }
 
     /**
-     * @param plc New segmentation policy.
-     */
-    public void policy(GridSegmentationPolicy plc) {
-        this.plc = plc;
-    }
-
-    /**
      * @return Segmentation resolvers.
      */
     @Nullable public String resolvers() {
@@ -86,13 +79,6 @@ public class VisorSegmentationConfiguration implements Serializable {
     }
 
     /**
-     * @param resolvers New segmentation resolvers.
-     */
-    public void resolvers(@Nullable String resolvers) {
-        this.resolvers = resolvers;
-    }
-
-    /**
      * @return Frequency of network segment check by discovery manager.
      */
     public long checkFrequency() {
@@ -100,13 +86,6 @@ public class VisorSegmentationConfiguration implements Serializable {
     }
 
     /**
-     * @param checkFreq New frequency of network segment check by discovery manager.
-     */
-    public void checkFrequency(long checkFreq) {
-        this.checkFreq = checkFreq;
-    }
-
-    /**
      * @return Whether or not node should wait for correct segment on start.
      */
     public boolean waitOnStart() {
@@ -114,26 +93,12 @@ public class VisorSegmentationConfiguration implements Serializable {
     }
 
     /**
-     * @param waitOnStart New whether or not node should wait for correct segment on start.
-     */
-    public void waitOnStart(boolean waitOnStart) {
-        this.waitOnStart = waitOnStart;
-    }
-
-    /**
      * @return Whether or not all resolvers should succeed for node to be in correct segment.
      */
     public boolean passRequired() {
         return passRequired;
     }
 
-    /**
-     * @param passRequired New whether or not all resolvers should succeed for node to be in correct segment.
-     */
-    public void passRequired(boolean passRequired) {
-        this.passRequired = passRequired;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorSegmentationConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSpisConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSpisConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSpisConfiguration.java
index 6833393..4afb90f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSpisConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorSpisConfiguration.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.visor.node;
 
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
@@ -141,16 +142,16 @@ public class VisorSpisConfiguration implements Serializable {
     public static VisorSpisConfiguration from(IgniteConfiguration c) {
         VisorSpisConfiguration cfg = new VisorSpisConfiguration();
 
-        cfg.discoverySpi(collectSpiInfo(c.getDiscoverySpi()));
-        cfg.communicationSpi(collectSpiInfo(c.getCommunicationSpi()));
-        cfg.eventStorageSpi(collectSpiInfo(c.getEventStorageSpi()));
-        cfg.collisionSpi(collectSpiInfo(c.getCollisionSpi()));
-        cfg.deploymentSpi(collectSpiInfo(c.getDeploymentSpi()));
-        cfg.checkpointSpis(collectSpiInfo(c.getCheckpointSpi()));
-        cfg.failoverSpis(collectSpiInfo(c.getFailoverSpi()));
-        cfg.loadBalancingSpis(collectSpiInfo(c.getLoadBalancingSpi()));
-        cfg.swapSpaceSpi(collectSpiInfo(c.getSwapSpaceSpi()));
-        cfg.indexingSpis(collectSpiInfo(c.getIndexingSpi()));
+        cfg.discoSpi = collectSpiInfo(c.getDiscoverySpi());
+        cfg.commSpi = collectSpiInfo(c.getCommunicationSpi());
+        cfg.evtSpi = collectSpiInfo(c.getEventStorageSpi());
+        cfg.colSpi = collectSpiInfo(c.getCollisionSpi());
+        cfg.deploySpi = collectSpiInfo(c.getDeploymentSpi());
+        cfg.cpSpis = collectSpiInfo(c.getCheckpointSpi());
+        cfg.failSpis = collectSpiInfo(c.getFailoverSpi());
+        cfg.loadBalancingSpis = collectSpiInfo(c.getLoadBalancingSpi());
+        cfg.swapSpaceSpis = collectSpiInfo(c.getSwapSpaceSpi());
+        cfg.indexingSpis = F.asArray(collectSpiInfo(c.getIndexingSpi()));
 
         return cfg;
     }
@@ -163,13 +164,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param discoSpi New discovery SPI.
-     */
-    public void discoverySpi(IgniteBiTuple<String, Map<String, Object>> discoSpi) {
-        this.discoSpi = discoSpi;
-    }
-
-    /**
      * @return Communication SPI.
      */
     public IgniteBiTuple<String, Map<String, Object>> communicationSpi() {
@@ -177,13 +171,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param commSpi New communication SPI.
-     */
-    public void communicationSpi(IgniteBiTuple<String, Map<String, Object>> commSpi) {
-        this.commSpi = commSpi;
-    }
-
-    /**
      * @return Event storage SPI.
      */
     public IgniteBiTuple<String, Map<String, Object>> eventStorageSpi() {
@@ -191,13 +178,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param evtSpi New event storage SPI.
-     */
-    public void eventStorageSpi(IgniteBiTuple<String, Map<String, Object>> evtSpi) {
-        this.evtSpi = evtSpi;
-    }
-
-    /**
      * @return Collision SPI.
      */
     public IgniteBiTuple<String, Map<String, Object>> collisionSpi() {
@@ -205,13 +185,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param colSpi New collision SPI.
-     */
-    public void collisionSpi(IgniteBiTuple<String, Map<String, Object>> colSpi) {
-        this.colSpi = colSpi;
-    }
-
-    /**
      * @return Deployment SPI.
      */
     public IgniteBiTuple<String, Map<String, Object>> deploymentSpi() {
@@ -219,13 +192,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param deploySpi New deployment SPI.
-     */
-    public void deploymentSpi(IgniteBiTuple<String, Map<String, Object>> deploySpi) {
-        this.deploySpi = deploySpi;
-    }
-
-    /**
      * @return Checkpoint SPIs.
      */
     public IgniteBiTuple<String, Map<String, Object>>[] checkpointSpis() {
@@ -233,13 +199,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param cpSpis New checkpoint SPIs.
-     */
-    public void checkpointSpis(IgniteBiTuple<String, Map<String, Object>>[] cpSpis) {
-        this.cpSpis = cpSpis;
-    }
-
-    /**
      * @return Failover SPIs.
      */
     public IgniteBiTuple<String, Map<String, Object>>[] failoverSpis() {
@@ -247,13 +206,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param failSpis New failover SPIs.
-     */
-    public void failoverSpis(IgniteBiTuple<String, Map<String, Object>>[] failSpis) {
-        this.failSpis = failSpis;
-    }
-
-    /**
      * @return Load balancing SPIs.
      */
     public IgniteBiTuple<String, Map<String, Object>>[] loadBalancingSpis() {
@@ -261,13 +213,6 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param loadBalancingSpis New load balancing SPIs.
-     */
-    public void loadBalancingSpis(IgniteBiTuple<String, Map<String, Object>>[] loadBalancingSpis) {
-        this.loadBalancingSpis = loadBalancingSpis;
-    }
-
-    /**
      * @return Swap space SPIs.
      */
     public IgniteBiTuple<String, Map<String, Object>> swapSpaceSpi() {
@@ -275,27 +220,12 @@ public class VisorSpisConfiguration implements Serializable {
     }
 
     /**
-     * @param swapSpaceSpis New swap space SPIs.
-     */
-    public void swapSpaceSpi(IgniteBiTuple<String, Map<String, Object>> swapSpaceSpis) {
-        this.swapSpaceSpis = swapSpaceSpis;
-    }
-
-    /**
      * @return Indexing SPIs.
      */
     public IgniteBiTuple<String, Map<String, Object>>[] indexingSpis() {
         return indexingSpis;
     }
 
-    /**
-     * @param indexingSpis New indexing SPIs.
-     */
-    @SafeVarargs
-    public final void indexingSpis(IgniteBiTuple<String, Map<String, Object>>... indexingSpis) {
-        this.indexingSpis = indexingSpis;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorSpisConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/57bdd8a5/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
index 597336b..667ff51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
@@ -3,7 +3,7 @@
  * 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 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
@@ -31,10 +31,10 @@ public class VisorTransactionConfiguration implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Default cache concurrency. */
-    private TransactionConcurrency dfltConcurrency;
+    private TransactionConcurrency dfltTxConcurrency;
 
     /** Default transaction isolation. */
-    private TransactionIsolation dfltIsolation;
+    private TransactionIsolation dfltTxIsolation;
 
     /** Default transaction timeout. */
     private long dfltTxTimeout;
@@ -57,12 +57,12 @@ public class VisorTransactionConfiguration implements Serializable {
     public static VisorTransactionConfiguration from(TransactionConfiguration src) {
         VisorTransactionConfiguration cfg = new VisorTransactionConfiguration();
 
-        cfg.defaultTxConcurrency(src.getDefaultTxConcurrency());
-        cfg.defaultTxIsolation(src.getDefaultTxIsolation());
-        cfg.defaultTxTimeout(src.getDefaultTxTimeout());
-        cfg.pessimisticTxLogLinger(src.getPessimisticTxLogLinger());
-        cfg.pessimisticTxLogSize(src.getPessimisticTxLogSize());
-        cfg.txSerializableEnabled(src.isTxSerializableEnabled());
+        cfg.dfltTxConcurrency = src.getDefaultTxConcurrency();
+        cfg.dfltTxIsolation = src.getDefaultTxIsolation();
+        cfg.dfltTxTimeout = src.getDefaultTxTimeout();
+        cfg.pessimisticTxLogLinger = src.getPessimisticTxLogLinger();
+        cfg.pessimisticTxLogSize = src.getPessimisticTxLogSize();
+        cfg.txSerEnabled = src.isTxSerializableEnabled();
 
         return cfg;
     }
@@ -71,28 +71,14 @@ public class VisorTransactionConfiguration implements Serializable {
      * @return Default cache transaction concurrency.
      */
     public TransactionConcurrency defaultTxConcurrency() {
-        return dfltConcurrency;
-    }
-
-    /**
-     * @param dfltConcurrency Default cache transaction concurrency.
-     */
-    public void defaultTxConcurrency(TransactionConcurrency dfltConcurrency) {
-        this.dfltConcurrency = dfltConcurrency;
+        return dfltTxConcurrency;
     }
 
     /**
      * @return Default cache transaction isolation.
      */
     public TransactionIsolation defaultTxIsolation() {
-        return dfltIsolation;
-    }
-
-    /**
-     * @param dfltIsolation Default cache transaction isolation.
-     */
-    public void defaultTxIsolation(TransactionIsolation dfltIsolation) {
-        this.dfltIsolation = dfltIsolation;
+        return dfltTxIsolation;
     }
 
     /**
@@ -103,13 +89,6 @@ public class VisorTransactionConfiguration implements Serializable {
     }
 
     /**
-     * @param dfltTxTimeout Default transaction timeout.
-     */
-    public void defaultTxTimeout(long dfltTxTimeout) {
-        this.dfltTxTimeout = dfltTxTimeout;
-    }
-
-    /**
      * @return Pessimistic log cleanup delay in milliseconds.
      */
     public int pessimisticTxLogLinger() {
@@ -117,13 +96,6 @@ public class VisorTransactionConfiguration implements Serializable {
     }
 
     /**
-     * @param pessimisticTxLogLinger Pessimistic log cleanup delay.
-     */
-    public void pessimisticTxLogLinger(int pessimisticTxLogLinger) {
-        this.pessimisticTxLogLinger = pessimisticTxLogLinger;
-    }
-
-    /**
      * @return Pessimistic transaction log size.
      */
     public int getPessimisticTxLogSize() {
@@ -131,26 +103,12 @@ public class VisorTransactionConfiguration implements Serializable {
     }
 
     /**
-     * @param pessimisticTxLogSize Pessimistic transactions log size.
-     */
-    public void pessimisticTxLogSize(int pessimisticTxLogSize) {
-        this.pessimisticTxLogSize = pessimisticTxLogSize;
-    }
-
-    /**
      * @return {@code True} if serializable transactions are enabled, {@code false} otherwise.
      */
     public boolean txSerializableEnabled() {
         return txSerEnabled;
     }
 
-    /**
-     * @param txSerEnabled Flag to enable/disable serializable cache transactions.
-     */
-    public void txSerializableEnabled(boolean txSerEnabled) {
-        this.txSerEnabled = txSerEnabled;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(VisorTransactionConfiguration.class, this);


[6/8] incubator-ignite git commit: # IGNITE-339 Review.

Posted by ak...@apache.org.
# IGNITE-339 Review.


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

Branch: refs/heads/sprint-2
Commit: 5d3a2be2b24b0d2c6e72648008219d269f92a142
Parents: 2710ece
Author: AKuznetsov <ak...@gridgain.com>
Authored: Sat Feb 28 01:53:32 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Sat Feb 28 01:53:32 2015 +0700

----------------------------------------------------------------------
 .../internal/visor/cache/VisorCacheConfiguration.java   | 12 ++----------
 .../visor/node/VisorTransactionConfiguration.java       |  2 +-
 .../ignite/visor/commands/cache/VisorCacheCommand.scala |  2 +-
 3 files changed, 4 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d3a2be2/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 8eee437..2ad0e49 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
@@ -93,7 +93,7 @@ public class VisorCacheConfiguration implements Serializable {
     /** Cache interceptor. */
     private String interceptor;
 
-    /** Should value bytes be stored. */
+    /** Flag indicating if cached values should be additionally stored in serialized form. */
     private boolean valBytes;
 
     /** Cache affinityCfg config. */
@@ -331,21 +331,13 @@ public class VisorCacheConfiguration implements Serializable {
     }
 
     /**
-     * @return Should value bytes be stored.
+     * @return {@code true} if cached values should be additionally stored in serialized form.
      */
     public boolean valueBytes() {
         return valBytes;
     }
 
     /**
-     * @param valBytes New should value bytes be stored.
-     */
-    public void valueBytes(boolean valBytes) {
-        this.valBytes = valBytes;
-    }
-
-
-    /**
      * @return Collection of type metadata.
      */
     public Collection<VisorCacheTypeMetadata> typeMeta() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d3a2be2/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
index 667ff51..773f9dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorTransactionConfiguration.java
@@ -3,7 +3,7 @@
  * 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 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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d3a2be2/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
index 577067a..7232221 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheCommand.scala
@@ -854,7 +854,7 @@ object VisorCacheCommand {
         cacheT += ("Concurrent Asynchronous Operations Number", cfg.maxConcurrentAsyncOperations())
         cacheT += ("Memory Mode", cfg.memoryMode())
 
-        cacheT += ("Store Values In Bytes", cfg.valueBytes())
+        cacheT += ("Store Values Bytes", cfg.valueBytes())
 
         cacheT += ("Off-Heap Size", cfg.offsetHeapMaxMemory())