You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2016/01/21 10:31:00 UTC

[1/7] ignite git commit: Fixed Visor tasks.

Repository: ignite
Updated Branches:
  refs/heads/sql-store 190182927 -> 50be0632a


Fixed Visor tasks.


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

Branch: refs/heads/sql-store
Commit: bccd459842f09572db1c0bf799508c732fa0bc30
Parents: c77fc84
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Jan 21 12:31:01 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Jan 21 12:31:01 2016 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCache.java | 45 ++++++++++----------
 .../internal/visor/cache/VisorCacheV2.java      | 23 +++++++---
 .../resources/META-INF/classnames.properties    | 15 ++++---
 3 files changed, 48 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bccd4598/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 5c1382b..0e830e2 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
@@ -257,35 +257,36 @@ public class VisorCache implements Serializable {
      * Fill values that should be stored in history;
      *
      * @param c Source cache.
+     * @return Cache.
      */
-    protected void initHistory(VisorCache c) {
-        c.name = name;
-        c.mode = mode;
-        c.memorySize = memorySize;
-        c.indexesSize = indexesSize;
-        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;
+    protected VisorCache initHistory(VisorCache c) {
+        if (c != null) {
+            c.name = name;
+            c.mode = mode;
+            c.memorySize = memorySize;
+            c.indexesSize = indexesSize;
+            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;
     }
 
     /**
      * @return New instance suitable to store in history.
      */
     public VisorCache history() {
-        VisorCache c = new VisorCache();
-
-        initHistory(c);
-
-        return c;
+        return initHistory(new VisorCache());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bccd4598/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
index a4b0409..6b6aba2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheV2.java
@@ -36,23 +36,32 @@ public class VisorCacheV2 extends VisorCache {
 
     /** {@inheritDoc} */
     @Override public VisorCache from(IgniteEx ignite, String cacheName, int sample) throws IgniteCheckedException {
-        super.from(ignite, cacheName, sample);
+        VisorCache c = super.from(ignite, cacheName, sample);
 
-        GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
+        if (c != null && c instanceof VisorCacheV2) {
+            GridCacheAdapter ca = ignite.context().cache().internalCache(cacheName);
 
-        // Cache was not started.
-        if (ca != null && ca.context().started())
-            near = ca.context().isNear();
+            // Cache was not started.
+            if (ca != null && ca.context().started())
+                ((VisorCacheV2)c).near = ca.context().isNear();
+        }
 
-        return this;
+        return c;
     }
 
     /** {@inheritDoc} */
-    @Override public void initHistory(VisorCache c) {
+    @Override protected VisorCache initHistory(VisorCache c) {
         super.initHistory(c);
 
         if (c instanceof VisorCacheV2)
             ((VisorCacheV2) c).near = near;
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override public VisorCache history() {
+        return initHistory(new VisorCacheV2());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bccd4598/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index d9a5514..8c3ad88 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -226,7 +226,6 @@ org.apache.ignite.internal.IgniteSchedulerImpl
 org.apache.ignite.internal.IgniteServicesImpl
 org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance$1
 org.apache.ignite.internal.NodeStoppingException
-org.apache.ignite.internal.binary.BinaryContext
 org.apache.ignite.internal.binary.BinaryEnumObjectImpl
 org.apache.ignite.internal.binary.BinaryMetadata
 org.apache.ignite.internal.binary.BinaryObjectEx
@@ -452,12 +451,12 @@ org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$6
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$7
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeFutureSet
 org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$MessageHandler
-org.apache.ignite.internal.processors.cache.GridCacheProcessor$2
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$1
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$3
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$4
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$5
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$6
-org.apache.ignite.internal.processors.cache.GridCacheProcessor$7
-org.apache.ignite.internal.processors.cache.GridCacheProcessor$9
+org.apache.ignite.internal.processors.cache.GridCacheProcessor$8
 org.apache.ignite.internal.processors.cache.GridCacheProcessor$LocalAffinityFunction
 org.apache.ignite.internal.processors.cache.GridCacheProxyImpl
 org.apache.ignite.internal.processors.cache.GridCacheReturn
@@ -496,7 +495,6 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$2
 org.apache.ignite.internal.processors.cache.GridCacheUtils$20
 org.apache.ignite.internal.processors.cache.GridCacheUtils$21
 org.apache.ignite.internal.processors.cache.GridCacheUtils$23
-org.apache.ignite.internal.processors.cache.GridCacheUtils$24
 org.apache.ignite.internal.processors.cache.GridCacheUtils$3
 org.apache.ignite.internal.processors.cache.GridCacheUtils$4
 org.apache.ignite.internal.processors.cache.GridCacheUtils$5
@@ -722,6 +720,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearGetResponse
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse
@@ -1113,7 +1112,7 @@ org.apache.ignite.internal.processors.platform.compute.PlatformBroadcastingMulti
 org.apache.ignite.internal.processors.platform.compute.PlatformBroadcastingSingleClosureTask
 org.apache.ignite.internal.processors.platform.compute.PlatformClosureJob
 org.apache.ignite.internal.processors.platform.compute.PlatformCompute$1
-org.apache.ignite.internal.processors.platform.compute.PlatformCompute$2
+org.apache.ignite.internal.processors.platform.compute.PlatformCompute$ComputeConvertingFuture$1
 org.apache.ignite.internal.processors.platform.compute.PlatformFullJob
 org.apache.ignite.internal.processors.platform.compute.PlatformFullTask
 org.apache.ignite.internal.processors.platform.compute.PlatformJob
@@ -1165,6 +1164,7 @@ org.apache.ignite.internal.processors.rest.GridRestProcessor$3
 org.apache.ignite.internal.processors.rest.GridRestResponse
 org.apache.ignite.internal.processors.rest.client.message.GridClientAbstractMessage
 org.apache.ignite.internal.processors.rest.client.message.GridClientAuthenticationRequest
+org.apache.ignite.internal.processors.rest.client.message.GridClientCacheBean
 org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest
 org.apache.ignite.internal.processors.rest.client.message.GridClientCacheRequest$GridCacheOperation
 org.apache.ignite.internal.processors.rest.client.message.GridClientHandshakeRequest
@@ -1548,6 +1548,7 @@ org.apache.ignite.internal.visor.cache.VisorCacheNearConfiguration
 org.apache.ignite.internal.visor.cache.VisorCacheNodesTask
 org.apache.ignite.internal.visor.cache.VisorCacheNodesTask$VisorCacheNodesJob
 org.apache.ignite.internal.visor.cache.VisorCacheQueryConfiguration
+org.apache.ignite.internal.visor.cache.VisorCacheQueryConfigurationV2
 org.apache.ignite.internal.visor.cache.VisorCacheQueryMetrics
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask
@@ -1560,10 +1561,12 @@ org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartJob
 org.apache.ignite.internal.visor.cache.VisorCacheStopTask
 org.apache.ignite.internal.visor.cache.VisorCacheStopTask$VisorCacheStopJob
 org.apache.ignite.internal.visor.cache.VisorCacheStoreConfiguration
+org.apache.ignite.internal.visor.cache.VisorCacheStoreConfigurationV2
 org.apache.ignite.internal.visor.cache.VisorCacheSwapBackupsTask
 org.apache.ignite.internal.visor.cache.VisorCacheSwapBackupsTask$VisorCachesSwapBackupsJob
 org.apache.ignite.internal.visor.cache.VisorCacheTypeFieldMetadata
 org.apache.ignite.internal.visor.cache.VisorCacheTypeMetadata
+org.apache.ignite.internal.visor.cache.VisorCacheV2
 org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionsTask
 org.apache.ignite.internal.visor.compute.VisorComputeCancelSessionsTask$VisorComputeCancelSessionsJob
 org.apache.ignite.internal.visor.compute.VisorComputeResetMetricsTask


[4/7] ignite git commit: Init simple value type if it was not set for a cache configuration received on discovery.

Posted by ag...@apache.org.
Init simple value type if it was not set for a cache configuration received on discovery.


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

Branch: refs/heads/sql-store
Commit: fb0173f3c7577cf36a60be3ab7e814cc301adb49
Parents: d2f84d1
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jan 21 12:17:43 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 21 12:17:43 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryProcessor.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fb0173f3/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 4808e96..64bbc8f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -318,7 +318,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     if (keyCls == null)
                         keyCls = Object.class;
 
-                    desc.name(meta.getSimpleValueType());
+                    String simpleValType = meta.getSimpleValueType();
+
+                    if (simpleValType == null)
+                        simpleValType = typeName(meta.getValueType());
+
+                    desc.name(simpleValType);
 
                     if (binaryEnabled && !keyOrValMustDeserialize) {
                         // Safe to check null.


[6/7] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by ag...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/sql-store
Commit: 7a31e4fb6d94a7141dfcb7c8bbde7cd8ce3165c3
Parents: abe44f9 fb0173f
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 21 12:20:29 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 21 12:20:29 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryProcessor.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[7/7] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into sql-store

Posted by ag...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into sql-store


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

Branch: refs/heads/sql-store
Commit: 50be0632a57bed3c6c3f4c5a746476f8c9160215
Parents: 1901829 7a31e4f
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jan 21 12:28:44 2016 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 21 12:28:44 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 18 ++++++++
 .../processors/query/GridQueryProcessor.java    |  7 ++-
 .../ignite/internal/visor/cache/VisorCache.java | 45 ++++++++++----------
 .../cache/VisorCacheQueryConfiguration.java     | 14 +++---
 .../cache/VisorCacheStoreConfiguration.java     | 26 ++++++-----
 .../internal/visor/cache/VisorCacheV2.java      | 23 +++++++---
 .../resources/META-INF/classnames.properties    | 15 ++++---
 7 files changed, 90 insertions(+), 58 deletions(-)
----------------------------------------------------------------------



[5/7] ignite git commit: Added initialization of 'storeKeepBinary' flag for cache configurations received on discovery.

Posted by ag...@apache.org.
Added initialization of 'storeKeepBinary' flag for cache configurations received on discovery.


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

Branch: refs/heads/sql-store
Commit: abe44f901722508de29abbf82bc5f3444e365290
Parents: d2f84d1
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 21 12:20:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 21 12:20:04 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 23 +++++++++++++++-----
 1 file changed, 17 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/abe44f90/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 7e165e0..13048ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1899,12 +1899,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
             else {
                 for (DynamicCacheChangeRequest req : batch.requests()) {
-                    if (req.startCacheConfiguration() != null) {
-                        CacheConfiguration ccfg = req.startCacheConfiguration();
-
-                        if (ccfg.isStoreKeepBinary() == null)
-                            ccfg.setStoreKeepBinary(CacheConfiguration.DFLT_STORE_KEEP_BINARY);
-                    }
+                    initReceivedCacheConfiguration(req);
 
                     if (req.template()) {
                         CacheConfiguration ccfg = req.startCacheConfiguration();
@@ -1997,6 +1992,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (DynamicCacheChangeRequest req : batch.requests()) {
             assert !req.template() : req;
 
+            initReceivedCacheConfiguration(req);
+
             String name = req.cacheName();
 
             boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
@@ -2445,6 +2442,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         boolean incMinorTopVer = false;
 
         for (DynamicCacheChangeRequest req : batch.requests()) {
+            initReceivedCacheConfiguration(req);
+
             if (req.template()) {
                 CacheConfiguration ccfg = req.startCacheConfiguration();
 
@@ -2589,6 +2588,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param req Cache change request.
+     */
+    private void initReceivedCacheConfiguration(DynamicCacheChangeRequest req) {
+        if (req.startCacheConfiguration() != null) {
+            CacheConfiguration ccfg = req.startCacheConfiguration();
+
+            if (ccfg.isStoreKeepBinary() == null)
+                ccfg.setStoreKeepBinary(CacheConfiguration.DFLT_STORE_KEEP_BINARY);
+        }
+    }
+
+    /**
      * Checks that preload-order-dependant caches has SYNC or ASYNC preloading mode.
      *
      * @param cfgs Caches.


[2/7] ignite git commit: Fixed Visor tasks.

Posted by ag...@apache.org.
Fixed Visor tasks.


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

Branch: refs/heads/sql-store
Commit: ba93717f9653769b43332b866c22aa69a58cfd0e
Parents: bccd459
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Jan 21 12:40:15 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Jan 21 12:40:15 2016 +0700

----------------------------------------------------------------------
 .../cache/VisorCacheQueryConfiguration.java     | 14 +++++------
 .../cache/VisorCacheStoreConfiguration.java     | 26 +++++++++-----------
 2 files changed, 18 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ba93717f/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
index c00d211..73088bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheQueryConfiguration.java
@@ -65,15 +65,13 @@ public class VisorCacheQueryConfiguration implements Serializable {
      * @return Fill data transfer object with cache query configuration data.
      */
     public VisorCacheQueryConfiguration from(CacheConfiguration ccfg) {
-        VisorCacheQueryConfiguration cfg = new VisorCacheQueryConfiguration();
+        sqlFuncClss = compactClasses(ccfg.getSqlFunctionClasses());
+        longQryWarnTimeout = ccfg.getLongQueryWarningTimeout();
+        sqlEscapeAll = ccfg.isSqlEscapeAll();
+        indexedTypes = compactClasses(ccfg.getIndexedTypes());
+        sqlOnheapRowCacheSize = ccfg.getSqlOnheapRowCacheSize();
 
-        cfg.sqlFuncClss = compactClasses(ccfg.getSqlFunctionClasses());
-        cfg.longQryWarnTimeout = ccfg.getLongQueryWarningTimeout();
-        cfg.sqlEscapeAll = ccfg.isSqlEscapeAll();
-        cfg.indexedTypes = compactClasses(ccfg.getIndexedTypes());
-        cfg.sqlOnheapRowCacheSize = ccfg.getSqlOnheapRowCacheSize();
-
-        return cfg;
+        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ba93717f/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 c152d76..2ba1b57 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
@@ -71,27 +71,25 @@ public class VisorCacheStoreConfiguration implements Serializable {
      * @return Data transfer object for cache store configuration properties.
      */
     public VisorCacheStoreConfiguration from(IgniteEx ignite, CacheConfiguration ccfg) {
-        VisorCacheStoreConfiguration cfg = new VisorCacheStoreConfiguration();
-
         IgniteCacheProxy<Object, Object> c = ignite.context().cache().jcache(ccfg.getName());
 
-        CacheStore store = c != null && c.context().started() ? c.context().store().configuredStore() : null;
+        CacheStore cstore = c != null && c.context().started() ? c.context().store().configuredStore() : null;
 
-        cfg.jdbcStore = store instanceof CacheAbstractJdbcStore;
+        jdbcStore = cstore instanceof CacheAbstractJdbcStore;
 
-        cfg.store = compactClass(store);
-        cfg.storeFactory = compactClass(ccfg.getCacheStoreFactory());
+        store = compactClass(cstore);
+        storeFactory = compactClass(ccfg.getCacheStoreFactory());
 
-        cfg.readThrough = ccfg.isReadThrough();
-        cfg.writeThrough = ccfg.isWriteThrough();
+        readThrough = ccfg.isReadThrough();
+        writeThrough = ccfg.isWriteThrough();
 
-        cfg.writeBehindEnabled = ccfg.isWriteBehindEnabled();
-        cfg.batchSz = ccfg.getWriteBehindBatchSize();
-        cfg.flushFreq = ccfg.getWriteBehindFlushFrequency();
-        cfg.flushSz = ccfg.getWriteBehindFlushSize();
-        cfg.flushThreadCnt = ccfg.getWriteBehindFlushThreadCount();
+        writeBehindEnabled = ccfg.isWriteBehindEnabled();
+        batchSz = ccfg.getWriteBehindBatchSize();
+        flushFreq = ccfg.getWriteBehindFlushFrequency();
+        flushSz = ccfg.getWriteBehindFlushSize();
+        flushThreadCnt = ccfg.getWriteBehindFlushThreadCount();
 
-        return cfg;
+        return this;
     }
 
     /**


[3/7] ignite git commit: Added initialization of 'storeKeepBinary' flag for cache configurations received on discovery.

Posted by ag...@apache.org.
Added initialization of 'storeKeepBinary' flag for cache configurations received on discovery.


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

Branch: refs/heads/sql-store
Commit: d2f84d1312e8bc22ad43b2e9076b881f0644d509
Parents: ba93717
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 21 11:45:27 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 21 11:45:27 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2f84d13/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 16dfa7f..7e165e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -1899,6 +1899,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
             else {
                 for (DynamicCacheChangeRequest req : batch.requests()) {
+                    if (req.startCacheConfiguration() != null) {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                        if (ccfg.isStoreKeepBinary() == null)
+                            ccfg.setStoreKeepBinary(CacheConfiguration.DFLT_STORE_KEEP_BINARY);
+                    }
+
                     if (req.template()) {
                         CacheConfiguration ccfg = req.startCacheConfiguration();