You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/05/05 14:24:07 UTC

[01/50] incubator-ignite git commit: # IGNITE-789 Review.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-834 1255e5849 -> 863d6bf69


# IGNITE-789 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/96f3c037
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/96f3c037
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/96f3c037

Branch: refs/heads/ignite-834
Commit: 96f3c0373b4e28c55dae1c6ff6cde1712ad78b70
Parents: c33d096
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 10:26:43 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 10:26:43 2015 +0700

----------------------------------------------------------------------
 .../internal/visor/cache/VisorCacheStartTask.java     | 14 +++++++++++---
 1 file changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/96f3c037/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
index 34f7b18..270c48a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
@@ -35,7 +35,8 @@ import java.util.*;
  * Task that start cache or near cache with specified configuration.
  */
 @GridInternal
-public class VisorCacheStartTask extends VisorMultiNodeTask<IgniteBiTuple<String, String>, Void, Void> {
+public class VisorCacheStartTask extends
+    VisorMultiNodeTask<IgniteBiTuple<String, String>, Map<UUID, IgniteException>, Void> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -44,8 +45,15 @@ public class VisorCacheStartTask extends VisorMultiNodeTask<IgniteBiTuple<String
         return new VisorCacheStartJob(arg, debug);
     }
 
-    @Nullable @Override protected Void reduce0(List<ComputeJobResult> results) throws IgniteException {
-        return null;
+    /** {@inheritDoc} */
+    @Nullable @Override protected Map<UUID, IgniteException> reduce0(List<ComputeJobResult> results) throws IgniteException {
+        Map<UUID, IgniteException> map = new HashMap<>();
+
+        for (ComputeJobResult res : results)
+            if (res.getException() != null)
+                map.put(res.getNode().id(), res.getException());
+
+        return map;
     }
 
     /**


[41/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 44b00ea46975ca8894ffbb24d53947ff4571d075
Parents: 04369fe 139aa27
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:51:39 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:51:39 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 349 insertions(+), 277 deletions(-)
----------------------------------------------------------------------



[32/50] incubator-ignite git commit: Merge branch 'ignite-728' into ignite-sprint-4

Posted by sb...@apache.org.
Merge branch 'ignite-728' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: d8ced90e512a71e8131329bdc37a936e88f55c3a
Parents: 2a176e4 65780ec5
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 15:40:30 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Apr 30 15:40:30 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 3 files changed, 115 insertions(+), 62 deletions(-)
----------------------------------------------------------------------



[21/50] incubator-ignite git commit: # ignite-791 Added note to DEVNOTES

Posted by sb...@apache.org.
# ignite-791 Added note to DEVNOTES


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

Branch: refs/heads/ignite-834
Commit: 73121f2e64ae169fcbdd6c4fcd1b22165367cb0c
Parents: 51bf584
Author: anovikov <an...@gridgain.com>
Authored: Thu Apr 30 00:07:21 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Thu Apr 30 00:07:21 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/73121f2e/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 2e37cd1..3542d97 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -9,7 +9,7 @@ With LGPL dependencies:
 Look for incubator-ignite-<version>-bin.zip in ./target/bin directory.
 
 NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
-
+      Schema Import module use JavaFX, under OpenJDK may be compiled and run only on OpenJDK 8 with installed OpenJFX.
 
 Ignite Hadoop Accelerator Maven Build Instructions
 ============================================


[20/50] incubator-ignite git commit: minor fix

Posted by sb...@apache.org.
minor fix


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

Branch: refs/heads/ignite-834
Commit: ef9b2d9a444bccee7ec070420b5194f6081db534
Parents: bba8c2f
Author: avinogradov <av...@gridgain.com>
Authored: Wed Apr 29 19:30:58 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Wed Apr 29 19:30:58 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ef9b2d9a/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 2e37cd1..ae37216 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -68,7 +68,7 @@ In case you want to release both fabric and hadoop editions you have to build ha
 restore them before deploy step.
 
 Start vote based on people.apache.org/~<username>/ignite-version.
-Close nexus staging when version accepted.
+Release nexus staging when version accepted.
 
 
 JCache TCK compliance


[34/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 71216191625c6d3f577c2e6e27bcb2c8f81d9a6b
Parents: 53996a1 d8ced90
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 15:55:15 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 15:55:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheTtlManager.java   | 168 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 3 files changed, 115 insertions(+), 62 deletions(-)
----------------------------------------------------------------------



[36/50] incubator-ignite git commit: Merge branches 'ignite-752' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-4

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


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

Branch: refs/heads/ignite-834
Commit: 49a992339a34e28b29dbbd84aa5dede1614eebb5
Parents: 7121619 e0648d6
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 17:22:06 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 17:22:06 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java      | 4 ----
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 ----
 .../ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java     | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 4 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[31/50] incubator-ignite git commit: Merge branch 'ignite-791' into ignite-sprint-4

Posted by sb...@apache.org.
Merge branch 'ignite-791' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 2a176e4987bc26a913d4e19bc939496b0079614f
Parents: ee5bae9 73121f2
Author: Andrey <an...@gridgain.com>
Authored: Thu Apr 30 17:10:58 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Apr 30 17:10:58 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt                       |   2 +-
 assembly/release-base.xml          |   2 +
 assembly/release-schema-import.xml |  50 ++++++++++++
 modules/schema-import/pom.xml      |   6 +-
 pom.xml                            | 137 +++++++++++++++++++++++++++-----
 5 files changed, 173 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2a176e49/DEVNOTES.txt
----------------------------------------------------------------------


[47/50] incubator-ignite git commit: Revert "#ignite-732: IgniteCache.size() should not fail in case of topology changes."

Posted by sb...@apache.org.
Revert "#ignite-732: IgniteCache.size() should not fail in case of topology changes."

This reverts commit 139aa270ae61494c0757867f2dc531ec7251b1da.


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

Branch: refs/heads/ignite-834
Commit: c4bc92974bace5e4cdb3ac9dc80790193e46d203
Parents: 281f4ef
Author: ivasilinets <iv...@gridgain.com>
Authored: Sat May 2 10:05:35 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Sat May 2 10:05:35 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 +++++++++----------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 277 insertions(+), 349 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index 87081fc..c2ad198 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -24,16 +24,15 @@ import java.util.*;
 
 /**
  * Convenience adapter for {@link ComputeTask} interface. Here is an example of
- * how {@code ComputeTaskAdapter} can be used:
+ * how {@code GridComputeTaskAdapter} can be used:
  * <pre name="code" class="java">
- * public class MyFooBarTask extends ComputeTaskAdapter&lt;String, String&gt; {
+ * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
  *     // Inject load balancer.
  *     &#64;LoadBalancerResource
  *     ComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg)
- *         throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -77,8 +76,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * <p>
      * If remote job resulted in exception ({@link ComputeJobResult#getException()} is not {@code null}),
      * then {@link ComputeJobResultPolicy#FAILOVER} policy will be returned if the exception is instance
-     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException},
-     * which means that remote node either failed or job execution was rejected before it got a chance to start. In all
+     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException}, which means that
+     * remote node either failed or job execution was rejected before it got a chance to start. In all
      * other cases the exception will be rethrown which will ultimately cause task to fail.
      *
      * @param res Received remote grid executable result.
@@ -88,8 +87,7 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * @throws IgniteException If handling a job result caused an error effectively rejecting
      *      a failover. This exception will be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
-        throws IgniteException {
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
         IgniteException e = res.getException();
 
         // Try to failover if result is failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 39f19b1..3f4e97b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -75,9 +75,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Failed result. */
-    private static final Object FAIL = new Integer(-1);
-
     /** clearLocally() split threshold. */
     public static final int CLEAR_ALL_SPLIT_THRESHOLD = 10000;
 
@@ -885,7 +882,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<Cache.Entry<K, V>> entrySet() {
-        return entrySet((CacheEntryPredicate[])null);
+        return entrySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
@@ -900,17 +897,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[])null);
+        return keySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
-        return primaryKeySet((CacheEntryPredicate[])null);
+        return primaryKeySet((CacheEntryPredicate[]) null);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<V> values() {
-        return values((CacheEntryPredicate[])null);
+        return values((CacheEntryPredicate[]) null);
     }
 
     /**
@@ -1083,31 +1080,36 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void clear() throws IgniteCheckedException {
-        clearAll(0, new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
+        // Clear local cache synchronously.
+        clearLocally();
+
+        clearRemotes(0, new GlobalClearAllCallable(name()));
     }
 
     /** {@inheritDoc} */
     @Override public void clear(K key) throws IgniteCheckedException {
-        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            Collections.singleton(key)));
+        // Clear local cache synchronously.
+        clearLocally(key);
+
+        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public void clearAll(Set<? extends K> keys) throws IgniteCheckedException {
-        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            keys));
+        // Clear local cache synchronously.
+        clearLocallyAll(keys);
+
+        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), keys));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(K key) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            Collections.singleton(key)));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(Set<? extends K> keys) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
-            keys));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), keys));
     }
 
     /**
@@ -1116,13 +1118,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @throws IgniteCheckedException In case of cache could not be cleared on any of the nodes.
      */
-    private void clearAll(long timeout, TopologyVersionAwareCallable clearCall) throws IgniteCheckedException {
+    private void clearRemotes(long timeout, GlobalClearCallable clearCall) throws IgniteCheckedException {
         try {
+            // Send job to remote nodes only.
+            Collection<ClusterNode> nodes =
+                ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes().nodes();
+
             IgniteInternalFuture<Object> fut = null;
 
-            ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
+            if (!nodes.isEmpty()) {
+                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
 
-            fut = new ClearFuture(ctx, clearCall);
+                fut = ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
+            }
 
             if (fut != null)
                 fut.get();
@@ -1141,18 +1149,19 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync() {
-        return clearAsync(new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
+        return clearAsync(new GlobalClearAllCallable(name()));
     }
 
     /**
      * @param clearCall Global clear callable object.
      * @return Future.
      */
-    private IgniteInternalFuture<?> clearAsync(TopologyVersionAwareCallable clearCall) {
+    private IgniteInternalFuture<?> clearAsync(GlobalClearCallable clearCall) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).nodes();
 
         if (!nodes.isEmpty()) {
-            IgniteInternalFuture<Object> fut = new ClearFuture(ctx, clearCall);
+            IgniteInternalFuture<Object> fut =
+                ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
 
             return fut.chain(new CX1<IgniteInternalFuture<Object>, Object>() {
                 @Override public Object applyx(IgniteInternalFuture<Object> fut) throws IgniteCheckedException {
@@ -2108,7 +2117,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 throws IgniteCheckedException {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys,
                     new C1<K, EntryProcessor<K, V, Object>>() {
-                        @Override public EntryProcessor apply(K k) {
+                            @Override public EntryProcessor apply(K k) {
                             return entryProcessor;
                         }
                     });
@@ -2136,7 +2145,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp() {
             @Override public IgniteInternalFuture<GridCacheReturn> inOp(IgniteTxLocalAdapter tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
-                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
+                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor);
 
                 return tx.invokeAsync(ctx, invokeMap, args);
             }
@@ -2362,7 +2371,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noValArray())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>) RET2VAL);
             }
 
             @Override public String toString() {
@@ -2517,7 +2526,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasValArray()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2906,7 +2915,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(oldVal);
 
-                return tx.putAllAsync(ctx,
+                return (GridCacheReturn) tx.putAllAsync(ctx,
                         F.t(key, newVal),
                         true,
                         null,
@@ -3008,7 +3017,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     ctx.deploy().registerClass(val);
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, false,
-                    ctx.equalsValArray(val)).get().success();
+                        ctx.equalsValArray(val)).get().success();
             }
 
             @Override public String toString() {
@@ -3221,10 +3230,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         TransactionConfiguration cfg = ctx.gridConfig().getTransactionConfiguration();
 
         return txStart(
-            concurrency,
-            isolation,
-            cfg.getDefaultTxTimeout(),
-            0
+                concurrency,
+                isolation,
+                cfg.getDefaultTxTimeout(),
+                0
         );
     }
 
@@ -3567,7 +3576,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
-        return new SizeFuture(peekModes, ctx, modes.near);
+        IgniteInternalFuture<Collection<Integer>> fut =
+            ctx.closures().broadcastNoFailover(new SizeCallable(ctx.name(), peekModes), null, nodes);
+
+        return fut.chain(new CX1<IgniteInternalFuture<Collection<Integer>>, Integer>() {
+            @Override public Integer applyx(IgniteInternalFuture<Collection<Integer>> fut)
+            throws IgniteCheckedException {
+                Collection<Integer> res = fut.get();
+
+                int totalSize = 0;
+
+                for (Integer size : res)
+                    totalSize += size;
+
+                return totalSize;
+            }
+        });
     }
 
     /** {@inheritDoc} */
@@ -3651,7 +3675,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return F.iterator(iterator(),
             new IgniteClosure<Cache.Entry<K, V>, Cache.Entry<K, V>>() {
                 private IgniteCacheExpiryPolicy expiryPlc =
-                    ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
+                        ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
 
                 @Override public Cache.Entry<K, V> apply(Cache.Entry<K, V> lazyEntry) {
                     CacheOperationContext prev = ctx.gate().enter(opCtx);
@@ -3885,6 +3909,50 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Gets cache global size (with or without backups).
+     *
+     * @param primaryOnly {@code True} if only primary sizes should be included.
+     * @return Global size.
+     * @throws IgniteCheckedException If internal task execution failed.
+     */
+    private int globalSize(boolean primaryOnly) throws IgniteCheckedException {
+        try {
+            // Send job to remote nodes only.
+            Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name()).forRemotes().nodes();
+
+            IgniteInternalFuture<Collection<Integer>> fut = null;
+
+            if (!nodes.isEmpty()) {
+                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
+
+                fut = ctx.closures().broadcastNoFailover(new GlobalSizeCallable(name(), primaryOnly), null, nodes);
+            }
+
+            // Get local value.
+            int globalSize = primaryOnly ? primarySize() : size();
+
+            if (fut != null) {
+                for (Integer i : fut.get())
+                    globalSize += i;
+            }
+
+            return globalSize;
+        }
+        catch (ClusterGroupEmptyCheckedException ignore) {
+            if (log.isDebugEnabled())
+                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
+
+            return primaryOnly ? primarySize() : size();
+        }
+        catch (ComputeTaskTimeoutCheckedException e) {
+            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
+                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
+
+            throw e;
+        }
+    }
+
+    /**
      * @param op Cache operation.
      * @param <T> Return type.
      * @return Operation result.
@@ -4825,10 +4893,47 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Internal callable which performs clear operation on a cache with the given name.
+     */
+    @GridInternal
+    private static abstract class GlobalClearCallable implements Callable<Object>, Externalizable {
+        /** Cache name. */
+        protected String cacheName;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        protected Ignite ignite;
+
+        /**
+         * Empty constructor for serialization.
+         */
+        public GlobalClearCallable() {
+            // No-op.
+        }
+
+        /**
+         * @param cacheName Cache name.
+         */
+        protected GlobalClearCallable(String cacheName) {
+            this.cacheName = cacheName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
+        }
+    }
+
+    /**
      * Global clear all.
      */
     @GridInternal
-    private static class GlobalClearAllCallable extends TopologyVersionAwareCallable {
+    private static class GlobalClearAllCallable extends GlobalClearCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4841,30 +4946,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          */
-        private GlobalClearAllCallable(String cacheName, AffinityTopologyVersion topVer) {
-            super(cacheName, topVer);
+        private GlobalClearAllCallable(String cacheName) {
+            super(cacheName);
         }
 
         /** {@inheritDoc} */
-        @Override protected Object callLocal() {
+        @Override public Object call() throws Exception {
             ((IgniteEx)ignite).cachex(cacheName).clearLocally();
 
             return null;
         }
-
-        /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
-        }
     }
 
     /**
      * Global clear keys.
      */
     @GridInternal
-    private static class GlobalClearKeySetCallable<K, V> extends TopologyVersionAwareCallable {
+    private static class GlobalClearKeySetCallable<K, V> extends GlobalClearCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4880,25 +4979,33 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          * @param keys Keys to clear.
          */
-        protected GlobalClearKeySetCallable(String cacheName, AffinityTopologyVersion topVer, Set<? extends K> keys) {
-            super(cacheName, topVer);
+        private GlobalClearKeySetCallable(String cacheName, Set<? extends K> keys) {
+            super(cacheName);
 
             this.keys = keys;
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
+        @Override public Object call() throws Exception {
+            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
+
+            return null;
         }
 
         /** {@inheritDoc} */
-        @Override protected Object callLocal() {
-            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            super.writeExternal(out);
 
-            return null;
+            out.writeObject(keys);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            super.readExternal(in);
+
+            keys = (Set<K>) in.readObject();
         }
     }
 
@@ -4906,202 +5013,127 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Internal callable for global size calculation.
      */
     @GridInternal
-    private static class GlobalSizeCallable extends TopologyVersionAwareCallable {
+    private static class SizeCallable extends IgniteClosureX<Object, Integer> implements Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** Cache name. */
+        private String cacheName;
+
         /** Peek modes. */
         private CachePeekMode[] peekModes;
 
-        /** Near enable. */
-        private boolean nearEnable;
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
 
         /**
          * Required by {@link Externalizable}.
          */
-        public GlobalSizeCallable() {
+        public SizeCallable() {
             // No-op.
         }
 
         /**
          * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
          * @param peekModes Cache peek modes.
          */
-        private GlobalSizeCallable(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes, boolean nearEnable) {
-            super(cacheName, topVer);
-
+        private SizeCallable(String cacheName, CachePeekMode[] peekModes) {
+            this.cacheName = cacheName;
             this.peekModes = peekModes;
-            this.nearEnable = nearEnable;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected Object callLocal() {
-            try {
-                IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
-
-                return cache == null ? 0 : cache.localSize(peekModes);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
         }
 
         /** {@inheritDoc} */
-        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
-            IgniteClusterEx cluster = ctx.grid().cluster();
+        @Override public Integer applyx(Object o) throws IgniteCheckedException {
+            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-            ClusterGroup grp = nearEnable ? cluster.forCacheNodes(ctx.name(), true, true, false) : cluster.forDataNodes(ctx.name());
+            assert cache != null : cacheName;
 
-            return grp.nodes();
+            return cache.localSize(peekModes);
         }
 
         /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(GlobalSizeCallable.class, this);
-        }
-    }
-
-    /**
-     * Cache size future.
-     */
-    private static class SizeFuture extends RetryFuture {
-        /** Size. */
-        private int size = 0;
-
-        /**
-         * @param peekModes Peek modes.
-         */
-        public SizeFuture(CachePeekMode[] peekModes, GridCacheContext ctx, boolean near) {
-            super(ctx, new GlobalSizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, near));
-        }
+        @SuppressWarnings("ForLoopReplaceableByForEach")
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
 
-        /** {@inheritDoc} */
-        @Override protected void onInit() {
-            size = 0;
-        }
+            out.writeInt(peekModes.length);
 
-        /** {@inheritDoc} */
-        @Override protected void onLocal(Object localRes) {
-            size += (Integer)localRes;
+            for (int i = 0; i < peekModes.length; i++)
+                U.writeEnum(out, peekModes[i]);
         }
 
         /** {@inheritDoc} */
-        @Override protected void allDone() {
-            onDone(size);
-        }
-    }
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
 
-    /**
-     * Cache clear future.
-     */
-    private static class ClearFuture extends RetryFuture {
-        /**
-         */
-        public ClearFuture(GridCacheContext ctx, TopologyVersionAwareCallable clearCall) {
-            super(ctx, clearCall);
-        }
+            int len = in.readInt();
 
-        /** {@inheritDoc} */
-        @Override protected void onInit() {
-           // No-op.
-        }
+            peekModes = new CachePeekMode[len];
 
-        /** {@inheritDoc} */
-        @Override protected void onLocal(Object localRes) {
-            // No-op.
+            for (int i = 0; i < len; i++)
+                peekModes[i] = CachePeekMode.fromOrdinal(in.readByte());
         }
 
         /** {@inheritDoc} */
-        @Override protected void allDone() {
-            onDone();
+        public String toString() {
+            return S.toString(SizeCallable.class, this);
         }
     }
 
     /**
-     * Retry future.
+     * Internal callable which performs {@link IgniteInternalCache#size()} or {@link IgniteInternalCache#primarySize()}
+     * operation on a cache with the given name.
      */
-    protected static abstract class RetryFuture<T> extends GridFutureAdapter<T> {
-        /** Context. */
-        private final GridCacheContext ctx;
+    @GridInternal
+    private static class GlobalSizeCallable implements IgniteClosure<Object, Integer>, Externalizable {
+        /** */
+        private static final long serialVersionUID = 0L;
 
-        /** Callable. */
-        private final TopologyVersionAwareCallable call;
+        /** Cache name. */
+        private String cacheName;
 
-        /** Max retries count before issuing an error. */
-        private volatile int retries = 32;
+        /** Primary only flag. */
+        private boolean primaryOnly;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
 
         /**
+         * Empty constructor for serialization.
          */
-        public RetryFuture(GridCacheContext ctx, TopologyVersionAwareCallable call) {
-            this.ctx = ctx;
-            this.call = call;
-
-            init();
+        public GlobalSizeCallable() {
+            // No-op.
         }
 
         /**
-         * Init.
+         * @param cacheName Cache name.
+         * @param primaryOnly Primary only flag.
          */
-        private void init() {
-            Collection<ClusterNode> nodes = call.nodes(ctx);
-
-            call.topologyVersion(ctx.affinity().affinityTopologyVersion());
-
-            IgniteInternalFuture<Collection<Object>> fut = ctx.closures().callAsyncNoFailover(BROADCAST,
-                F.asSet((Callable<Object>)call), nodes, true);
-
-            fut.listen(new IgniteInClosure<IgniteInternalFuture<Collection<Object>>>() {
-                @Override public void apply(IgniteInternalFuture<Collection<Object>> fut) {
-                    try {
-                        Collection<Object> res = fut.get();
-
-                        onInit();
-
-                        for (Object locRes : res) {
-                            if (locRes == FAIL) {
-                                if (retries-- > 0)
-                                    init();
-                                else {
-                                    onDone(new ClusterTopologyException("Failed to wait topology."));
-
-                                    return;
-                                }
-                            }
+        private GlobalSizeCallable(String cacheName, boolean primaryOnly) {
+            this.cacheName = cacheName;
+            this.primaryOnly = primaryOnly;
+        }
 
-                            onLocal(locRes);
-                        }
+        /** {@inheritDoc} */
+        @Override public Integer apply(Object o) {
+            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-                        allDone();
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (X.hasCause(e, ClusterTopologyException.class)) {
-                            if (retries-- > 0)
-                                init();
-                            else
-                                onDone(e);
-                        }
-                        else
-                            onDone(e);
-                    }
-                }
-            });
+            return primaryOnly ? cache.primarySize() : cache.size();
         }
 
-        /**
-         * Init reducer.
-         */
-        protected abstract void onInit();
-
-        /**
-         * @param localRes Add local result to global result.
-         */
-        protected abstract void onLocal(Object localRes);
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            U.writeString(out, cacheName);
+            out.writeBoolean(primaryOnly);
+        }
 
-        /**
-         * On done.
-         */
-        protected abstract void allDone();
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            cacheName = U.readString(in);
+            primaryOnly = in.readBoolean();
+        }
     }
 
     /**
@@ -5665,89 +5697,4 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             metrics.addPutAndGetTimeNanos(System.nanoTime() - start);
         }
     }
-
-    /**
-     * Delayed callable class.
-     */
-    protected static abstract class TopologyVersionAwareCallable<K, V> implements Serializable, Callable<Object> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        protected Ignite ignite;
-
-        /** Affinity topology version. */
-        protected AffinityTopologyVersion topVer;
-
-        /** Cache name. */
-        protected String cacheName;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public TopologyVersionAwareCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param topVer Affinity topology version.
-         */
-        public TopologyVersionAwareCallable(String cacheName, AffinityTopologyVersion topVer) {
-            this.cacheName = cacheName;
-            this.topVer = topVer;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            if (!compareTopologyVersions())
-                return FAIL;
-
-            Object res = callLocal();
-
-            if (!compareTopologyVersions())
-                return FAIL;
-            else
-                return res;
-        }
-
-        /**
-         * Call local.
-         *
-         * @return Local result.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected abstract Object callLocal() throws IgniteCheckedException;
-
-        /**
-         * @param ctx Grid cache context.
-         * @return Nodes to call.
-         */
-        protected abstract Collection<ClusterNode> nodes(GridCacheContext ctx);
-
-        /**
-         * Compare topology versions.
-         */
-        public boolean compareTopologyVersions() {
-            GridCacheProcessor cacheProc = ((IgniteKernal) ignite).context().cache();
-
-            GridCacheAdapter<K, V> cacheAdapter = cacheProc.internalCache(cacheName);
-
-            if (cacheAdapter == null)
-                return false;
-
-            final GridCacheContext<K, V> ctx = cacheAdapter.context();
-
-            AffinityTopologyVersion locTopVer = ctx.affinity().affinityTopologyVersion();
-
-            return locTopVer.compareTo(topVer) == 0;
-        }
-
-        /**
-         * @param topVer Affinity topology version.
-         */
-        public void topologyVersion(AffinityTopologyVersion topVer) {
-            this.topVer = topVer;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c4bc9297/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 77fa104..c0026ab 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
@@ -124,9 +124,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Must use JDK marshaller since it is used by discovery to fire custom events. */
     private Marshaller marshaller = new JdkMarshaller();
 
-    /** Count down latch for caches. */
-    private CountDownLatch cacheStartedLatch = new CountDownLatch(1);
-
     /**
      * @param ctx Kernal context.
      */
@@ -660,92 +657,87 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        try {
-            if (ctx.config().isDaemon())
-                return;
+        if (ctx.config().isDaemon())
+            return;
 
-            ClusterNode locNode = ctx.discovery().localNode();
+        ClusterNode locNode = ctx.discovery().localNode();
 
-            // Init cache plugin managers.
-            final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
+        // Init cache plugin managers.
+        final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
 
-            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                CacheConfiguration locCcfg = desc.cacheConfiguration();
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            CacheConfiguration locCcfg = desc.cacheConfiguration();
 
-                CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
+            CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
 
-                cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
-            }
+            cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
+        }
 
-            if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
-                for (ClusterNode n : ctx.discovery().remoteNodes()) {
-                    checkTransactionConfiguration(n);
+        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+            for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                checkTransactionConfiguration(n);
 
-                    DeploymentMode locDepMode = ctx.config().getDeploymentMode();
-                    DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
+                DeploymentMode locDepMode = ctx.config().getDeploymentMode();
+                DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
-                    CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
-                        locDepMode, rmtDepMode, true);
+                CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
+                    locDepMode, rmtDepMode, true);
 
-                    for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                        CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
+                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                    CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
 
-                        if (rmtCfg != null) {
-                            CacheConfiguration locCfg = desc.cacheConfiguration();
+                    if (rmtCfg != null) {
+                        CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n);
+                        checkCache(locCfg, rmtCfg, n);
 
-                            // Check plugin cache configurations.
-                            CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
+                        // Check plugin cache configurations.
+                        CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
 
-                            assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                        assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                            pluginMgr.validateRemotes(rmtCfg, n);
-                        }
+                        pluginMgr.validateRemotes(rmtCfg, n);
                     }
                 }
             }
+        }
 
-            // Start dynamic caches received from collect discovery data.
-            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                boolean started = desc.onStart();
+        // Start dynamic caches received from collect discovery data.
+        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+            boolean started = desc.onStart();
 
-                assert started : "Failed to change started flag for locally configured cache: " + desc;
+            assert started : "Failed to change started flag for locally configured cache: " + desc;
 
-                desc.clearRemoteConfigurations();
+            desc.clearRemoteConfigurations();
 
-                CacheConfiguration ccfg = desc.cacheConfiguration();
+            CacheConfiguration ccfg = desc.cacheConfiguration();
 
-                IgnitePredicate filter = ccfg.getNodeFilter();
+            IgnitePredicate filter = ccfg.getNodeFilter();
 
-                if (filter.apply(locNode)) {
-                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+            if (filter.apply(locNode)) {
+                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-                    CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
+                CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
 
-                    assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
 
-                    ctx.dynamicDeploymentId(desc.deploymentId());
+                ctx.dynamicDeploymentId(desc.deploymentId());
 
-                    sharedCtx.addCacheContext(ctx);
+                sharedCtx.addCacheContext(ctx);
 
-                    GridCacheAdapter cache = ctx.cache();
+                GridCacheAdapter cache = ctx.cache();
 
-                    String name = ccfg.getName();
+                String name = ccfg.getName();
 
-                    caches.put(maskNull(name), cache);
+                caches.put(maskNull(name), cache);
 
-                    startCache(cache);
+                startCache(cache);
 
-                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
-                }
+                jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
             }
         }
-        finally {
-            cacheStartedLatch.countDown();
-        }
 
         ctx.marshallerContext().onMarshallerCacheStarted(ctx);
 
@@ -843,8 +835,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStop(boolean cancel) {
-        cacheStartedLatch.countDown();
-
         if (ctx.config().isDaemon())
             return;
 
@@ -2696,13 +2686,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting internal cache adapter: " + name);
 
-        try {
-            cacheStartedLatch.await();
-        }
-        catch (InterruptedException e) {
-            throw new IgniteException("Failed to wait starting caches.");
-        }
-
         return (GridCacheAdapter<K, V>)caches.get(maskNull(name));
     }
 


[10/50] incubator-ignite git commit: IGNITE-789 classnames

Posted by sb...@apache.org.
IGNITE-789 classnames


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

Branch: refs/heads/ignite-834
Commit: 98aa8f1f242a6d841a304a2c958d0bdd48ed1bbd
Parents: a06f5fa
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 20:38:23 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 20:38:23 2015 +0700

----------------------------------------------------------------------
 .../src/main/resources/META-INF/classnames.properties  | 13 +++++++++++++
 1 file changed, 13 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/98aa8f1f/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 a79d5b8..35495ed 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -115,6 +115,7 @@ org.apache.ignite.configuration.CollectionConfiguration
 org.apache.ignite.configuration.DeploymentMode
 org.apache.ignite.configuration.IgniteReflectionFactory
 org.apache.ignite.configuration.NearCacheConfiguration
+org.apache.ignite.configuration.TopologyValidator
 org.apache.ignite.configuration.TransactionConfiguration
 org.apache.ignite.events.CacheEvent
 org.apache.ignite.events.CacheQueryExecutedEvent
@@ -248,6 +249,7 @@ org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMa
 org.apache.ignite.internal.managers.communication.GridIoMessage
 org.apache.ignite.internal.managers.communication.GridIoPolicy
 org.apache.ignite.internal.managers.communication.GridIoUserMessage
+org.apache.ignite.internal.managers.communication.GridLifecycleAwareMessageFilter
 org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean
 org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2
 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest
@@ -445,6 +447,7 @@ org.apache.ignite.internal.processors.cache.IgniteCacheProxy$2
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$4
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$5
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$6
+org.apache.ignite.internal.processors.cache.IgniteCacheProxy$7
 org.apache.ignite.internal.processors.cache.KeyCacheObject
 org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl
 org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy
@@ -452,6 +455,7 @@ org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresMa
 org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager$QueueHeaderPredicate
 org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager$RemoveSetDataCallable
 org.apache.ignite.internal.processors.cache.distributed.GridCacheCommittedTxInfo
+org.apache.ignite.internal.processors.cache.distributed.GridCacheOptimisticCheckPreparedTxFuture$1
 org.apache.ignite.internal.processors.cache.distributed.GridCacheOptimisticCheckPreparedTxRequest
 org.apache.ignite.internal.processors.cache.distributed.GridCacheOptimisticCheckPreparedTxResponse
 org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest
@@ -725,6 +729,7 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$11
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4
@@ -762,9 +767,11 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$AtomicInt
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$NodeFailureTimeoutObject$1
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxMap
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxMap$1
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxMap$1$1
@@ -809,6 +816,7 @@ org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$Discove
 org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$DiscoveryDataItem
 org.apache.ignite.internal.processors.continuous.GridContinuousProcessor$StartRequestData
 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$3
+org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$4
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Batched
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$BatchedSorted
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Individual
@@ -1383,6 +1391,8 @@ org.apache.ignite.internal.visor.cache.VisorCacheMetrics
 org.apache.ignite.internal.visor.cache.VisorCacheMetricsCollectorTask
 org.apache.ignite.internal.visor.cache.VisorCacheMetricsCollectorTask$VisorCacheMetricsCollectorJob
 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.VisorCacheQueryMetrics
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration
@@ -1390,6 +1400,9 @@ org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob
 org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask
 org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob
+org.apache.ignite.internal.visor.cache.VisorCacheStartTask
+org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartArg
+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


[24/50] incubator-ignite git commit: # Removed marshaller warning

Posted by sb...@apache.org.
# Removed marshaller warning


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

Branch: refs/heads/ignite-834
Commit: c8f7d387e1c52fb11bc890009b1d94292a533508
Parents: 3344b09
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Apr 29 11:50:31 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Apr 29 11:50:31 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgnitionEx.java    | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c8f7d387/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 9453791..8d88677 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1788,11 +1788,6 @@ public class IgnitionEx {
                 else
                     marsh = new OptimizedMarshaller();
             }
-            else if (marsh instanceof OptimizedMarshaller && !U.isHotSpot()) {
-                U.warn(log, "Using OptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
-                    "object serialization behavior could yield unexpected results.",
-                    "Using GridOptimizedMarshaller on untested JVM.");
-            }
 
             myCfg.setMarshaller(marsh);
 


[29/50] incubator-ignite git commit: Merge branch 'ignite-sprint-4' into ignite-728

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' into ignite-728


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

Branch: refs/heads/ignite-834
Commit: 65780ec532bcae5e45946aef70b210e3ca6e26a8
Parents: 35e90b8 ee5bae9
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 12:23:55 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Apr 30 12:23:55 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 .../configuration/CacheConfiguration.java       | 255 +++++++++++---
 .../configuration/IgniteConfiguration.java      | 344 +++++++++++++++----
 .../ignite/internal/GridUpdateNotifier.java     |  66 +++-
 .../apache/ignite/internal/IgniteKernal.java    |  83 +++--
 .../org/apache/ignite/internal/IgnitionEx.java  |  15 +-
 .../managers/communication/GridIoManager.java   |   8 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  13 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 ---------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 12 files changed, 762 insertions(+), 353 deletions(-)
----------------------------------------------------------------------



[14/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 47136b5fe9a4afa31f6ca4cb6909fcabb28c9200
Parents: b01ff38 5854f1d
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 21:36:19 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 21:36:19 2015 +0700

----------------------------------------------------------------------
 modules/cloud/pom.xml                             | 4 +---
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/gce/pom.xml                               | 4 +---
 pom.xml                                           | 3 +--
 4 files changed, 4 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[46/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the withAsync method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the withAsync method.


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

Branch: refs/heads/ignite-834
Commit: 281f4ef208e7928c8514992458f34ab96781f671
Parents: 2cb7317
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:50:18 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:50:18 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/281f4ef2/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
index f5b5153..be05a38 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
@@ -24,7 +24,7 @@ public interface IgniteAsyncSupport {
     /**
      * Gets instance of this component with asynchronous mode enabled.
      *
-     * @return Component with asynchronous mode enabled.
+     * @return Instance of this component with asynchronous mode enabled.
      */
     public IgniteAsyncSupport withAsync();
 


[23/50] incubator-ignite git commit: ignite-327: enable OptimizedMarshaller for another JDKs

Posted by sb...@apache.org.
ignite-327: enable OptimizedMarshaller for another JDKs


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

Branch: refs/heads/ignite-834
Commit: 3344b09b2ceb9e7489fbb32ee3648832e698bd69
Parents: ef9b2d9
Author: Artem Shutak <as...@gridgain.com>
Authored: Wed Apr 29 20:55:54 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Wed Apr 29 20:55:54 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/internal/IgnitionEx.java  | 10 +---------
 1 file changed, 1 insertion(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3344b09b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 58a8c8d..9453791 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1775,15 +1775,7 @@ public class IgnitionEx {
             Marshaller marsh = myCfg.getMarshaller();
 
             if (marsh == null) {
-                if (!U.isHotSpot()) {
-                    U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
-                        "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
-                        "object serialization performance will be significantly slower.",
-                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
-
-                    marsh = new JdkMarshaller();
-                }
-                else if (!OptimizedMarshaller.available()) {
+                if (!OptimizedMarshaller.available()) {
                     U.warn(log, "OptimizedMarshaller is not supported on this JVM " +
                         "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
                         "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +


[42/50] incubator-ignite git commit: src-release zip fix

Posted by sb...@apache.org.
src-release zip fix


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

Branch: refs/heads/ignite-834
Commit: 09a33fe39b734f43767fd8ed9bbe2f1275f18324
Parents: 44b00ea
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Apr 30 23:32:41 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Apr 30 23:32:41 2015 +0300

----------------------------------------------------------------------
 pom.xml | 39 ++++++++++++++++++++++++++++++++++-----
 1 file changed, 34 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/09a33fe3/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98541ef..c859021 100644
--- a/pom.xml
+++ b/pom.xml
@@ -467,6 +467,35 @@
             <build>
                 <plugins>
                     <plugin>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.apache.resources</groupId>
+                                <artifactId>apache-source-release-assembly-descriptor</artifactId>
+                                <version>1.0.4</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <id>source-release-assembly</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <runOnlyAtExecutionRoot>true</runOnlyAtExecutionRoot>
+                                    <descriptorRefs>
+                                        <descriptorRef>${sourceReleaseAssemblyDescriptor}</descriptorRef>
+                                    </descriptorRefs>
+                                    <tarLongFileMode>gnu</tarLongFileMode>
+                                    <finalName>incubator-ignite-${project.version}-src</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+
+                    <plugin>
                         <groupId>org.apache.maven.plugins</groupId>
                         <artifactId>maven-antrun-plugin</artifactId>
                         <executions>
@@ -533,7 +562,7 @@
                                 <fileSet>
                                     <directory>${basedir}/target</directory>
                                     <includes>
-                                        <include>${project.artifactId}-${project.version}-source-release.zip</include>
+                                        <include>incubator-ignite-${project.version}-src.zip</include>
                                         <include>bin/*.zip</include>
                                     </includes>
                                 </fileSet>
@@ -561,10 +590,10 @@
                                     <failOnError>false</failOnError>
                                     <target>
                                         <mkdir dir="${basedir}/target/site" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.asc" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.asc" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.md5" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.md5" failonerror="false" />
-                                        <copy file="${basedir}/target/${project.artifactId}-${project.version}-source-release.zip.sha1" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.sha1" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.asc" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.asc" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.md5" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.md5" failonerror="false" />
+                                        <copy file="${basedir}/target/incubator-ignite-${project.version}-src.zip.sha1" tofile="${basedir}/target/site/incubator-ignite-${project.version}-src.zip.sha1" failonerror="false" />
                                         <copy todir="${basedir}/target/site">
                                             <fileset dir="${basedir}/target/bin">
                                                 <include name="**/*" />


[25/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-838' into ignite-sprint-4

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-838' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 1e533950ffcbb1ec16781ff406a654cfd612e1cc
Parents: c8f7d38 dbd8d64
Author: sboikov <se...@inria.fr>
Authored: Wed Apr 29 21:59:51 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Wed Apr 29 21:59:51 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/IgniteSpiAdapter.java  | 19 ++++++++++++++-----
 1 file changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[33/50] incubator-ignite git commit: Apache release fix

Posted by sb...@apache.org.
Apache release fix


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

Branch: refs/heads/ignite-834
Commit: 53996a12effd51bfb5049f8dd42f391d0f2d9d19
Parents: 2a176e4
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 15:54:50 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 15:54:50 2015 +0300

----------------------------------------------------------------------
 pom.xml | 51 +++++++++++++++++++++++++++++++++++++++++++++++++++
 1 file changed, 51 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/53996a12/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 03600e6..98541ef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -37,6 +37,7 @@
 
     <properties>
         <ignite.site>scp://localhost:/home</ignite.site>
+        <!--fix <attachartifact>...</> at apache-release profile if changed-->
         <ignite.zip.pattern>ignite-${ignite.edition}-${project.version}-incubating</ignite.zip.pattern>
     </properties>
 
@@ -462,6 +463,55 @@
         </profile>
 
         <profile>
+            <id>apache-release</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-antrun-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <id>attach-artifact</id>
+                                <phase>package</phase>
+                                <goals>
+                                    <goal>run</goal>
+                                </goals>
+                                <configuration>
+                                    <failOnError>false</failOnError>
+                                    <target>
+                                        <attachartifact
+                                            file="${basedir}/target/bin/ignite-fabric-${project.version}-incubating.zip"
+                                            classifier="fabric"
+                                            type="zip"/>
+                                        <attachartifact
+                                            file="${basedir}/target/bin/ignite-hadoop-${project.version}-incubating.zip"
+                                            classifier="hadoop"
+                                            type="zip"/>
+                                    </target>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-gpg-plugin</artifactId>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>sign</goal>
+                                </goals>
+                                <configuration>
+                                    <ascDirectory>${basedir}</ascDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+        <profile>
             <id>deploy-ignite-site</id>
             <build>
                 <plugins>
@@ -484,6 +534,7 @@
                                     <directory>${basedir}/target</directory>
                                     <includes>
                                         <include>${project.artifactId}-${project.version}-source-release.zip</include>
+                                        <include>bin/*.zip</include>
                                     </includes>
                                 </fileSet>
                             </fileSets>


[11/50] incubator-ignite git commit: Merge branch 'ignite-789' into ignite-sprint-4

Posted by sb...@apache.org.
Merge branch 'ignite-789' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: d1a84a576f5a07d89fbc190a7365b6e328ff5353
Parents: edcf921 98aa8f1
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 20:56:10 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 20:56:10 2015 +0700

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   4 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheStartTask.java        | 155 +++++++++++++++++++
 .../internal/visor/util/VisorTaskUtils.java     |  10 ++
 .../resources/META-INF/classnames.properties    |  13 ++
 .../commands/cache/VisorCacheCommand.scala      |   2 +-
 6 files changed, 183 insertions(+), 5 deletions(-)
----------------------------------------------------------------------



[06/50] incubator-ignite git commit: # IGNITE-839 Fixed cache metrics.

Posted by sb...@apache.org.
# IGNITE-839 Fixed cache metrics.


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

Branch: refs/heads/ignite-834
Commit: db8333d77d68669edc203de8e69ad49ecc62f4c0
Parents: a7599bf
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 17:42:43 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 17:42:43 2015 +0700

----------------------------------------------------------------------
 .../internal/visor/cache/VisorCacheMetrics.java | 53 +++++++-------------
 1 file changed, 19 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/db8333d7/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
index c05b9e0..30be424 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
@@ -29,7 +29,7 @@ import java.io.*;
  */
 public class VisorCacheMetrics implements Serializable {
     /** */
-    private static final int MICROSECONDS_IN_SECOND = 1_000_000;
+    private static final float MICROSECONDS_IN_SECOND = 1_000_000;
 
     /** */
     private static final long serialVersionUID = 0L;
@@ -94,14 +94,11 @@ public class VisorCacheMetrics implements Serializable {
     /** Reads per second. */
     private int readsPerSec;
 
-    /** Writes per second. */
-    private int writesPerSec;
+    /** Puts per second. */
+    private int putsPerSec;
 
-    /** Hits per second. */
-    private int hitsPerSec;
-
-    /** Misses per second. */
-    private int missesPerSec;
+    /** Removes per second. */
+    private int removalsPerSec;
 
     /** Commits per second. */
     private int commitsPerSec;
@@ -160,15 +157,11 @@ public class VisorCacheMetrics implements Serializable {
     /**
      * Calculate rate of metric per second.
      *
-     * @param metric Metric value.
-     * @param time Metric finish time.
-     * @param createTime Metric start time.
+     * @param meanTime Metric mean time.
      * @return Metric per second.
      */
-    private static int perSecond(int metric, long time, long createTime) {
-        long seconds = (time - createTime) / 1000;
-
-        return (seconds > 0) ? (int)(metric / seconds) : 0;
+    private static int perSecond(float meanTime) {
+        return (meanTime > 0) ? (int)(MICROSECONDS_IN_SECOND / meanTime) : 0;
     }
 
     /**
@@ -209,12 +202,11 @@ public class VisorCacheMetrics implements Serializable {
         cm.avgPutTime = m.getAveragePutTime();
         cm.avgRemovalTime = m.getAverageRemoveTime();
 
-        cm.readsPerSec = (int)(MICROSECONDS_IN_SECOND * 1.f / m.getAverageGetTime());
-        cm.writesPerSec = (int)(MICROSECONDS_IN_SECOND * 1.f / m.getAveragePutTime());
-        cm.hitsPerSec = -1;
-        cm.missesPerSec = (int)(MICROSECONDS_IN_SECOND * 1.f / m.getAverageRemoveTime());
-        cm.commitsPerSec = (int)(MICROSECONDS_IN_SECOND * 1.f / m.getAverageTxCommitTime());
-        cm.rollbacksPerSec = (int)(MICROSECONDS_IN_SECOND * 1.f / m.getAverageTxRollbackTime());
+        cm.readsPerSec = perSecond(m.getAverageGetTime());
+        cm.putsPerSec = perSecond(m.getAveragePutTime());
+        cm.removalsPerSec = perSecond(m.getAverageRemoveTime());
+        cm.commitsPerSec = perSecond(m.getAverageTxCommitTime());
+        cm.rollbacksPerSec = perSecond(m.getAverageTxRollbackTime());
 
         cm.qryMetrics = VisorCacheQueryMetrics.from(c.context().queries().metrics());
 
@@ -364,24 +356,17 @@ public class VisorCacheMetrics implements Serializable {
     }
 
     /**
-     * @return Writes per second.
-     */
-    public int writesPerSecond() {
-        return writesPerSec;
-    }
-
-    /**
-     * @return Hits per second.
+     * @return Puts per second.
      */
-    public int hitsPerSecond() {
-        return hitsPerSec;
+    public int putsPerSecond() {
+        return putsPerSec;
     }
 
     /**
-     * @return Misses per second.
+     * @return Removes per second.
      */
-    public int missesPerSecond() {
-        return missesPerSec;
+    public int removalsPerSecond() {
+        return removalsPerSec;
     }
 
     /**


[43/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the close method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the close method.


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

Branch: refs/heads/ignite-834
Commit: f298d7216a81b0416c1cf123efd99fcff2d41e19
Parents: 09a33fe
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:28:44 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:28:44 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/examples/streaming/wordcount/CacheConfig.java | 5 -----
 modules/core/src/main/java/org/apache/ignite/IgniteCache.java   | 5 +++++
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f298d721/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java b/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
index bb2a18e..58704ca 100644
--- a/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
+++ b/examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/CacheConfig.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.examples.streaming.wordcount;
 
 import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.configuration.*;
 
 import javax.cache.configuration.*;
@@ -44,10 +43,6 @@ public class CacheConfig {
         // Sliding window of 1 seconds.
         cfg.setExpiryPolicyFactory(FactoryBuilder.factoryOf(new CreatedExpiryPolicy(new Duration(SECONDS, 1))));
 
-        // Do not allow more than 1 million entries.
-        // Allows to run this example with smaller available memory.
-        cfg.setEvictionPolicy(new FifoEvictionPolicy<>(1_000_000));
-
         return cfg;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f298d721/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index d99902a..bf4080a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -499,6 +499,11 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
         CacheEntryProcessor<K, V, T> entryProcessor, Object... args);
 
     /**
+     * Completely deletes the cache with all its data from the system.
+     */
+    @Override void close();
+
+    /**
      * This cache node to re-balance its partitions. This method is usually used when
      * {@link CacheConfiguration#getRebalanceDelay()} configuration parameter has non-zero value.
      * When many nodes are started or stopped almost concurrently, it is more efficient to delay


[16/50] incubator-ignite git commit: Merge branch 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-728

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-728


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

Branch: refs/heads/ignite-834
Commit: d59e4c42fbf3a55afbba9dae3d9e81c9f13e98fa
Parents: 8ae6b03 47136b5
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:01:14 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:01:14 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++
 modules/cloud/pom.xml                           | 106 ++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 +++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 ++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../configuration/CacheConfiguration.java       |   4 +-
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |  28 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 ++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 ++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../visor/cache/VisorCacheStartTask.java        | 155 ++++++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |  12 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../distributed/GridCacheLockAbstractTest.java  |  75 +++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 +++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 +++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 +++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 ++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 +++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 ++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 ++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 +++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 ++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 +++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 ++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 ++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 +++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 ++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 ++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 +++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 +++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 ++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  36 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 439 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 275 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 182 files changed, 8304 insertions(+), 3154 deletions(-)
----------------------------------------------------------------------



[50/50] incubator-ignite git commit: #ignite-834: Change size and clear to jobs.

Posted by sb...@apache.org.
#ignite-834: Change size and clear to jobs.


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

Branch: refs/heads/ignite-834
Commit: 863d6bf69f75c2bf1a7571ffa76259544baf0539
Parents: 086553e
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue May 5 15:23:41 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue May 5 15:23:41 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 331 +++++--------------
 .../GridDistributedCacheAdapter.java            |  28 +-
 2 files changed, 103 insertions(+), 256 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/863d6bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index d59e9cc..596705e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -25,7 +25,6 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
-import org.apache.ignite.internal.compute.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
@@ -1123,32 +1122,31 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @throws IgniteCheckedException In case of cache could not be cleared on any of the nodes.
      */
-    private void clearRemotes(long timeout, GlobalClearCallable clearCall) throws IgniteCheckedException {
-        try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes =
-                ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes().nodes();
+    private void clearRemotes(long timeout, final TopologyVersionAwareJob clearCall) throws IgniteCheckedException {
+        // Send job to remote nodes only.
+        ClusterGroup nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes();
 
-            IgniteInternalFuture<Object> fut = null;
+        if (!nodes.nodes().isEmpty()) {
+            ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
 
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
+            ctx.grid().context().task().setThreadContext(TC_SUBGRID, nodes);
 
-                fut = ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
-            }
+            ctx.grid().context().task().execute(new ComputeTaskAdapter<Object, Object>() {
+                /** {@inheritDoc} */
+                @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+                    @Nullable Object arg) throws IgniteException {
+                    Map<ComputeJob, ClusterNode> jobs = new HashMap();
+                    for (ClusterNode node : subgrid)
+                        jobs.put(clearCall, node);
 
-            if (fut != null)
-                fut.get();
-        }
-        catch (ClusterGroupEmptyCheckedException ignore) {
-            if (log.isDebugEnabled())
-                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
-        }
-        catch (ComputeTaskTimeoutCheckedException e) {
-            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
-                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
+                    return jobs;
+                }
 
-            throw e;
+                /** {@inheritDoc} */
+                @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteException {
+                    return null;
+                }
+            }, null).get();
         }
     }
 
@@ -1161,26 +1159,28 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @return Future.
      */
-    private IgniteInternalFuture<?> clearAsync(GlobalClearCallable clearCall) {
+    private IgniteInternalFuture<?> clearAsync(final TopologyVersionAwareJob clearCall) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).nodes();
 
         if (!nodes.isEmpty()) {
-            IgniteInternalFuture<Object> fut =
-                ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
+            ctx.grid().context().task().setThreadContext(TC_SUBGRID, nodes);
 
-            return fut.chain(new CX1<IgniteInternalFuture<Object>, Object>() {
-                @Override public Object applyx(IgniteInternalFuture<Object> fut) throws IgniteCheckedException {
-                    try {
-                        return fut.get();
-                    }
-                    catch (ClusterGroupEmptyCheckedException ignore) {
-                        if (log.isDebugEnabled())
-                            log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
+            return ctx.grid().context().task().execute(new ComputeTaskAdapter<Object, Object>() {
+                /** {@inheritDoc} */
+                @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+                                                                  @Nullable Object arg) throws IgniteException {
+                    Map<ComputeJob, ClusterNode> jobs = new HashMap();
+                    for (ClusterNode node : subgrid)
+                        jobs.put(clearCall, node);
 
-                        return null;
-                    }
+                    return jobs;
                 }
-            });
+
+                /** {@inheritDoc} */
+                @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteException {
+                    return null;
+                }
+            }, null);
         }
         else
             return new GridFinishedFuture<>();
@@ -3567,7 +3567,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Integer> sizeAsync(CachePeekMode[] peekModes) {
+    @Override public IgniteInternalFuture<Integer> sizeAsync(final CachePeekMode[] peekModes) {
         assert peekModes != null;
 
         PeekModes modes = parsePeekModes(peekModes, true);
@@ -3581,23 +3581,32 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
-        IgniteInternalFuture<Collection<Integer>> fut =
-            ctx.closures().broadcastNoFailover(
-                new SizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), null, nodes);
+        ctx.grid().context().task().setThreadContext(TC_SUBGRID, nodes);
+
+        return ctx.grid().context().task().execute(new ComputeTaskAdapter<Object, Integer>() {
+
+            /** {@inheritDoc} */
+            @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+                @Nullable Object arg) throws IgniteException {
+                Map<ComputeJob, ClusterNode> jobs = new HashMap();
+                for (ClusterNode node : subgrid)
+                    jobs.put(new SizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes), node);
 
-        return fut.chain(new CX1<IgniteInternalFuture<Collection<Integer>>, Integer>() {
-            @Override public Integer applyx(IgniteInternalFuture<Collection<Integer>> fut)
-            throws IgniteCheckedException {
-                Collection<Integer> res = fut.get();
+                return jobs;
+            }
 
-                int totalSize = 0;
+            /** {@inheritDoc} */
+            @Nullable @Override public Integer reduce(List<ComputeJobResult> results) throws IgniteException {
+                int size = 0;
 
-                for (Integer size : res)
-                    totalSize += size;
+                for (ComputeJobResult res : results) {
+                    if (res.getException() == null)
+                        size += res.<Integer>getData();
+                }
 
-                return totalSize;
+                return size;
             }
-        });
+        }, null);
     }
 
     /** {@inheritDoc} */
@@ -3915,51 +3924,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Gets cache global size (with or without backups).
-     *
-     * @param primaryOnly {@code True} if only primary sizes should be included.
-     * @return Global size.
-     * @throws IgniteCheckedException If internal task execution failed.
-     */
-    private int globalSize(boolean primaryOnly) throws IgniteCheckedException {
-        try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name()).forRemotes().nodes();
-
-            IgniteInternalFuture<Collection<Integer>> fut = null;
-
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
-
-                fut = ctx.closures().broadcastNoFailover(
-                    new GlobalSizeCallable(name(), ctx.affinity().affinityTopologyVersion(), primaryOnly), null, nodes);
-            }
-
-            // Get local value.
-            int globalSize = primaryOnly ? primarySize() : size();
-
-            if (fut != null) {
-                for (Integer i : fut.get())
-                    globalSize += i;
-            }
-
-            return globalSize;
-        }
-        catch (ClusterGroupEmptyCheckedException ignore) {
-            if (log.isDebugEnabled())
-                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
-
-            return primaryOnly ? primarySize() : size();
-        }
-        catch (ComputeTaskTimeoutCheckedException e) {
-            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
-                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
-
-            throw e;
-        }
-    }
-
-    /**
      * @param op Cache operation.
      * @param <T> Return type.
      * @return Operation result.
@@ -4900,41 +4864,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Internal callable which performs clear operation on a cache with the given name.
-     */
-    @GridInternal
-    private static abstract class GlobalClearCallable extends DelayedCallable implements Callable<Object> {
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalClearCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
-         */
-        protected GlobalClearCallable(String cacheName, AffinityTopologyVersion topVer) {
-            super(cacheName, topVer);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-        }
-    }
-
-    /**
      * Global clear all.
      */
     @GridInternal
-    private static class GlobalClearAllCallable extends GlobalClearCallable {
+    private static class GlobalClearAllCallable extends TopologyVersionAwareJob {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4954,9 +4887,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            waitAffinityReadyFuture();
-
+        @Nullable @Override public Object localExecute() {
             ((IgniteEx)ignite).cachex(cacheName).clearLocally();
 
             return null;
@@ -4967,7 +4898,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Global clear keys.
      */
     @GridInternal
-    private static class GlobalClearKeySetCallable<K, V> extends GlobalClearCallable {
+    private static class GlobalClearKeySetCallable<K, V> extends TopologyVersionAwareJob {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4993,34 +4924,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            waitAffinityReadyFuture();
-
+        @Nullable @Override public Object localExecute() {
             ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
 
             return null;
         }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            out.writeObject(keys);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-
-            keys = (Set<K>) in.readObject();
-        }
     }
 
     /**
      * Internal callable for global size calculation.
      */
     @GridInternal
-    private static class SizeCallable extends DelayedCallable implements IgniteClosure<Object, Integer> {
+    private static class SizeCallable extends TopologyVersionAwareJob {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -5046,9 +4961,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @Override public Integer apply(Object o) {
-            waitAffinityReadyFuture();
-
+        @Nullable @Override public Object localExecute() {
             IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
             assert cache != null : cacheName;
@@ -5062,89 +4975,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings("ForLoopReplaceableByForEach")
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            out.writeInt(peekModes.length);
-
-            for (int i = 0; i < peekModes.length; i++)
-                U.writeEnum(out, peekModes[i]);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-
-            int len = in.readInt();
-
-            peekModes = new CachePeekMode[len];
-
-            for (int i = 0; i < len; i++)
-                peekModes[i] = CachePeekMode.fromOrdinal(in.readByte());
-        }
-
-        /** {@inheritDoc} */
         public String toString() {
             return S.toString(SizeCallable.class, this);
         }
     }
 
     /**
-     * Internal callable which performs {@link IgniteInternalCache#size()} or {@link IgniteInternalCache#primarySize()}
-     * operation on a cache with the given name.
-     */
-    @GridInternal
-    private static class GlobalSizeCallable extends DelayedCallable implements IgniteClosure<Object, Integer>, Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Primary only flag. */
-        private boolean primaryOnly;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalSizeCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param cacheName Cache name.
-         * @param topVer Affinity topology version.
-         * @param primaryOnly Primary only flag.
-         */
-        private GlobalSizeCallable(String cacheName, AffinityTopologyVersion topVer, boolean primaryOnly) {
-            super(cacheName, topVer);
-
-            this.primaryOnly = primaryOnly;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer apply(Object o) {
-            waitAffinityReadyFuture();
-
-            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
-
-            return primaryOnly ? cache.primarySize() : cache.size();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            out.writeBoolean(primaryOnly);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-
-            primaryOnly = in.readBoolean();
-        }
-    }
-
-    /**
      * Holder for last async operation future.
      */
     protected static class FutureHolder {
@@ -5709,7 +5545,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * Delayed callable class.
      */
-    protected static abstract class DelayedCallable<K, V> implements Externalizable {
+    protected static abstract class TopologyVersionAwareJob<K, V> extends ComputeJobAdapter {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -5730,22 +5566,37 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         /**
          * Empty constructor for serialization.
          */
-        public DelayedCallable() {
+        public TopologyVersionAwareJob() {
             // No-op.
         }
 
         /**
+         * @param cacheName Cache name.
          * @param topVer Affinity topology version.
          */
-        public DelayedCallable(String cacheName, AffinityTopologyVersion topVer) {
+        public TopologyVersionAwareJob(String cacheName, AffinityTopologyVersion topVer) {
             this.cacheName = cacheName;
             this.topVer = topVer;
         }
 
+        /** {@inheritDoc} */
+        @Nullable @Override public final Object execute() {
+            waitAffinityReadyFuture();
+
+            localExecute();
+
+            return null;
+        }
+
+        /**
+         * @return Local execution result.
+         */
+        @Nullable protected abstract Object localExecute();
+
         /**
          * Hold (suspend) job execution until our cache version becomes equal to remote cache's version.
          */
-        public void waitAffinityReadyFuture() {
+        private void waitAffinityReadyFuture() {
             GridCacheProcessor cacheProc = ((IgniteKernal) ignite).context().cache();
 
             GridCacheAdapter<K, V> cacheAdapter = cacheProc.internalCache(cacheName);
@@ -5768,21 +5619,5 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 }
             }
         }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
-
-            topVer.writeExternal(out);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
-
-            topVer = new AffinityTopologyVersion();
-
-            topVer.readExternal(in);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/863d6bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 661df87..3a685cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
@@ -230,13 +231,23 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
      * operation on a cache with the given name.
      */
     @GridInternal
-    private static class GlobalRemoveAllCallable<K,V> extends DelayedCallable<K, V> implements Callable<Object> {
+    private static class GlobalRemoveAllCallable<K,V> implements Callable<Object>, Externalizable {
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** Cache name. */
+        private String cacheName;
+
+        /** Topology version. */
+        private AffinityTopologyVersion topVer;
+
         /** Skip store flag. */
         private boolean skipStore;
 
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
         /**
          * Empty constructor for serialization.
          */
@@ -250,7 +261,8 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
          * @param skipStore Skip store flag.
          */
         private GlobalRemoveAllCallable(String cacheName, @NotNull AffinityTopologyVersion topVer, boolean skipStore) {
-            super(cacheName, topVer);
+            this.cacheName = cacheName;
+            this.topVer = topVer;
             this.skipStore = skipStore;
         }
 
@@ -258,11 +270,11 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
          * {@inheritDoc}
          */
         @Override public Object call() throws Exception {
-            GridCacheAdapter<K, V> cacheAdapter = ((IgniteKernal) ignite).context().cache().internalCache(cacheName);
+            GridCacheAdapter<K, V> cacheAdapter = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
 
             final GridCacheContext<K, V> ctx = cacheAdapter.context();
 
-            waitAffinityReadyFuture();
+            ctx.affinity().affinityReadyFuture(topVer).get();
 
             ctx.gate().enter();
 
@@ -326,15 +338,15 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
 
         /** {@inheritDoc} */
         @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
+            U.writeString(out, cacheName);
+            out.writeObject(topVer);
             out.writeBoolean(skipStore);
         }
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
-
+            cacheName = U.readString(in);
+            topVer = (AffinityTopologyVersion)in.readObject();
             skipStore = in.readBoolean();
         }
     }


[48/50] incubator-ignite git commit: IGNITE-856 - gg.client -> ignite.client

Posted by sb...@apache.org.
IGNITE-856 - gg.client -> ignite.client


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

Branch: refs/heads/ignite-834
Commit: 54f949220433d45b543de83896ed3e357a68e04f
Parents: c4bc929
Author: Ignite Teamcity <ig...@apache.org>
Authored: Mon May 4 17:27:14 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Mon May 4 17:27:14 2015 -0700

----------------------------------------------------------------------
 .../config/grid-client-config.properties        | 50 ++++++------
 .../ClientPropertiesConfigurationSelfTest.java  | 12 +--
 .../org/apache/ignite/IgniteJdbcDriver.java     | 81 ++++++++++----------
 .../client/GridClientConfiguration.java         |  2 +-
 4 files changed, 71 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/clients/config/grid-client-config.properties
----------------------------------------------------------------------
diff --git a/modules/clients/config/grid-client-config.properties b/modules/clients/config/grid-client-config.properties
index b6bce8d..d25352b 100644
--- a/modules/clients/config/grid-client-config.properties
+++ b/modules/clients/config/grid-client-config.properties
@@ -16,89 +16,89 @@
 #
 
 # Required. Comma-separated list of servers to connect to in format "host:port".
-gg.client.servers=localhost:11211
+ignite.client.servers=localhost:11211
 
 # Optional. Default client load balancer. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.balancer=
+#ignite.client.balancer=
 
 # Optional. Socket connect timeout (ms). Default value is "0".
-#gg.client.connectTimeout=10000
+#ignite.client.connectTimeout=10000
 
 # Optional. Credentials if grid requires authentication. Default value is "".
-#gg.client.credentials=
+#ignite.client.credentials=
 
 # Optional. Flag indicating whether topology cache is enabled. Default value is "false".
-#gg.client.cacheTop=false
+#ignite.client.cacheTop=false
 
 # Optional. Max time of connection idleness (ms). Default value is "30000".
-#gg.client.idleTimeout=30000
+#ignite.client.idleTimeout=30000
 
 # Optional. Client protocol, one of TCP or HTTP. Default value is "TCP".
-#gg.client.protocol=TCP
+#ignite.client.protocol=TCP
 
 # Optional. TCP_NODELAY communication flag. Default value is "true".
-#gg.client.tcp.noDelay=true
+#ignite.client.tcp.noDelay=true
 
 # Optional. Topology refresh frequency (ms). Default value is "2000".
-#gg.client.topology.refresh=2000
+#ignite.client.topology.refresh=2000
 
 #
 # Data configurations.
 #
 
 # Optional. List of comma-separated names of data configurations. Default value is "".
-#gg.client.data.configurations=cfg1, cfg2
+#ignite.client.data.configurations=cfg1, cfg2
 
 # Optional. Cache name (space) to work with (for configuration 'cfg1').
-#gg.client.data.cfg1.name=
+#ignite.client.data.cfg1.name=
 
 # Optional. Specific load balancer for configuration 'cfg1'. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.data.cfg1.balancer=random
+#ignite.client.data.cfg1.balancer=random
 
 # Optional. Specific affinity for configuration 'cfg1'. Default value is "".
 # Affinity may be "" (no affinity), "partitioned" or full class name for "your.affinity.ImplementationClass"
-#gg.client.data.cfg1.affinity=
+#ignite.client.data.cfg1.affinity=
 
 # Optional. Cache name (space) to work with (for configuration 'cfg2').
-#gg.client.data.cfg2.name=partitioned
+#ignite.client.data.cfg2.name=partitioned
 
 # Optional. Specific load balancer for configuration 'cfg2'. Default value is "".
 # Balancer may be "random", "roundrobin" or full class name for "your.balancer.ImplementationClass"
-#gg.client.data.cfg2.balancer=roundrobin
+#ignite.client.data.cfg2.balancer=roundrobin
 
 # Optional. Specific affinity for configuration 'cfg2'. Default value is "".
 # Affinity may be "" (no affinity), "partitioned" or full class name for "your.affinity.ImplementationClass"
-#gg.client.data.cfg2.affinity=partitioned
+#ignite.client.data.cfg2.affinity=partitioned
 
 #
 # SSL configuration.
 #
 
 # Optional. SSL enabled. Default value is "false".
-#gg.client.ssl.enabled=false
+#ignite.client.ssl.enabled=false
 
 # Optional. SSL protocol. Default value is "TLS".
-#gg.client.ssl.protocol=TLS
+#ignite.client.ssl.protocol=TLS
 
 # Optional. Key manager algorithm. Default value is "SunX509".
-#gg.client.ssl.key.algorithm=SunX509
+#ignite.client.ssl.key.algorithm=SunX509
 
 # Optional. Keystore to be used by client to connect with Ignite topology over SSL. Default value is "".
-#gg.client.ssl.keystore.location=
+#ignite.client.ssl.keystore.location=
 
 # Optional. Default value is "".
-#gg.client.ssl.keystore.password=
+#ignite.client.ssl.keystore.password=
 
 # Optional. Default value is "jks".
-#gg.client.ssl.keystore.type=jks
+#ignite.client.ssl.keystore.type=jks
 
 # Optional. Truststore to be used by client to connect with Ignite topology over SSL. Default value is "".
-#gg.client.ssl.truststore.location=
+#ignite.client.ssl.truststore.location=
 
 # Optional. Default value is "".
-#gg.client.ssl.truststore.password=
+#ignite.client.ssl.truststore.password=
 
 # Optional. Default value is "jks".
-#gg.client.ssl.truststore.type=jks
+#ignite.client.ssl.truststore.type=jks

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
index 6e2a1eb..8e981d1 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/impl/ClientPropertiesConfigurationSelfTest.java
@@ -85,8 +85,8 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         for (Map.Entry<Object, Object> e : props.entrySet())
             props2.put("new." + e.getKey(), e.getValue());
 
-        validateConfig(0, new GridClientConfiguration("new.gg.client", props2));
-        validateConfig(0, new GridClientConfiguration("new.gg.client.", props2));
+        validateConfig(0, new GridClientConfiguration("new.ignite.client", props2));
+        validateConfig(0, new GridClientConfiguration("new.ignite.client.", props2));
 
         // Validate loaded test configuration.
         File tmp = uncommentProperties(GRID_CLIENT_CONFIG);
@@ -100,14 +100,14 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         for (Map.Entry<Object, Object> e : props.entrySet())
             props2.put("new." + e.getKey(), e.getValue());
 
-        validateConfig(2, new GridClientConfiguration("new.gg.client", props2));
-        validateConfig(2, new GridClientConfiguration("new.gg.client.", props2));
+        validateConfig(2, new GridClientConfiguration("new.ignite.client", props2));
+        validateConfig(2, new GridClientConfiguration("new.ignite.client.", props2));
 
         // Validate loaded test configuration with empty key prefixes.
         props2 = new Properties();
 
         for (Map.Entry<Object, Object> e : props.entrySet())
-            props2.put(e.getKey().toString().replace("gg.client.", ""), e.getValue());
+            props2.put(e.getKey().toString().replace("ignite.client.", ""), e.getValue());
 
         validateConfig(2, new GridClientConfiguration("", props2));
         validateConfig(2, new GridClientConfiguration(".", props2));
@@ -156,7 +156,7 @@ public class ClientPropertiesConfigurationSelfTest extends GridCommonAbstractTes
         Collection<String> lines = new ArrayList<>();
 
         while (it.hasNext())
-            lines.add(it.nextLine().replace("#gg.client.", "gg.client."));
+            lines.add(it.nextLine().replace("#ignite.client.", "ignite.client."));
 
         IgniteUtils.closeQuiet(in);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
index a7c2eae..17ec221 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcDriver.java
@@ -90,101 +90,101 @@ import java.util.logging.*;
  *         <th>Optional</th>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.protocol</b></td>
+ *         <td><b>ignite.client.protocol</b></td>
  *         <td>Communication protocol ({@code TCP} or {@code HTTP}).</td>
  *         <td>{@code TCP}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.connectTimeout</b></td>
+ *         <td><b>ignite.client.connectTimeout</b></td>
  *         <td>Socket connection timeout.</td>
  *         <td>{@code 0} (infinite timeout)</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.tcp.noDelay</b></td>
+ *         <td><b>ignite.client.tcp.noDelay</b></td>
  *         <td>Flag indicating whether TCP_NODELAY flag should be enabled for outgoing connections.</td>
  *         <td>{@code true}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.enabled</b></td>
+ *         <td><b>ignite.client.ssl.enabled</b></td>
  *         <td>Flag indicating that {@code SSL} is needed for connection.</td>
  *         <td>{@code false}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.protocol</b></td>
+ *         <td><b>ignite.client.ssl.protocol</b></td>
  *         <td>SSL protocol ({@code SSL} or {@code TLS}).</td>
  *         <td>{@code TLS}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.key.algorithm</b></td>
+ *         <td><b>ignite.client.ssl.key.algorithm</b></td>
  *         <td>Key manager algorithm.</td>
  *         <td>{@code SunX509}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.location</b></td>
+ *         <td><b>ignite.client.ssl.keystore.location</b></td>
  *         <td>Key store to be used by client to connect with Ignite topology.</td>
  *         <td>&nbsp;</td>
  *         <td>No (if {@code SSL} is enabled)</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.password</b></td>
+ *         <td><b>ignite.client.ssl.keystore.password</b></td>
  *         <td>Key store password.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.keystore.type</b></td>
+ *         <td><b>ignite.client.ssl.keystore.type</b></td>
  *         <td>Key store type.</td>
  *         <td>{@code jks}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.location</b></td>
+ *         <td><b>ignite.client.ssl.truststore.location</b></td>
  *         <td>Trust store to be used by client to connect with Ignite topology.</td>
  *         <td>&nbsp;</td>
  *         <td>No (if {@code SSL} is enabled)</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.password</b></td>
+ *         <td><b>ignite.client.ssl.truststore.password</b></td>
  *         <td>Trust store password.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.ssl.truststore.type</b></td>
+ *         <td><b>ignite.client.ssl.truststore.type</b></td>
  *         <td>Trust store type.</td>
  *         <td>{@code jks}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.credentials</b></td>
+ *         <td><b>ignite.client.credentials</b></td>
  *         <td>Client credentials used in authentication process.</td>
  *         <td>&nbsp;</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.cache.top</b></td>
+ *         <td><b>ignite.client.cache.top</b></td>
  *         <td>
  *             Flag indicating that topology is cached internally. Cache will be refreshed in
- *             the background with interval defined by {@code gg.client.topology.refresh}
+ *             the background with interval defined by {@code ignite.client.topology.refresh}
  *             property (see below).
  *         </td>
  *         <td>{@code false}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.topology.refresh</b></td>
+ *         <td><b>ignite.client.topology.refresh</b></td>
  *         <td>Topology cache refresh frequency (ms).</td>
  *         <td>{@code 2000}</td>
  *         <td>Yes</td>
  *     </tr>
  *     <tr>
- *         <td><b>gg.client.idleTimeout</b></td>
+ *         <td><b>ignite.client.idleTimeout</b></td>
  *         <td>Maximum amount of time that connection can be idle before it is closed (ms).</td>
  *         <td>{@code 30000}</td>
  *         <td>Yes</td>
@@ -225,7 +225,7 @@ import java.util.logging.*;
 @SuppressWarnings("JavadocReference")
 public class IgniteJdbcDriver implements Driver {
     /** Prefix for property names. */
-    private static final String PROP_PREFIX = "gg.jdbc.";
+    private static final String PROP_PREFIX = "ignite.jdbc.";
 
     /** Hostname property name. */
     public static final String PROP_HOST = PROP_PREFIX + "host";
@@ -236,11 +236,8 @@ public class IgniteJdbcDriver implements Driver {
     /** Cache name property name. */
     public static final String PROP_CACHE = PROP_PREFIX + "cache";
 
-    /** Node ID URL parameter name. */
-    public static final String PARAM_NODE_ID = "nodeId";
-
     /** Node ID property name. */
-    public static final String PROP_NODE_ID = PROP_PREFIX + PARAM_NODE_ID;
+    public static final String PROP_NODE_ID = PROP_PREFIX + "nodeId";
 
     /** URL prefix. */
     public static final String URL_PREFIX = "jdbc:ignite://";
@@ -290,40 +287,40 @@ public class IgniteJdbcDriver implements Driver {
         props[1] = new PropertyInfo("Port number", info.getProperty(PROP_PORT), "");
         props[2] = new PropertyInfo("Cache name", info.getProperty(PROP_CACHE), "");
         props[3] = new PropertyInfo("Node ID", info.getProperty(PROP_NODE_ID, ""));
-        props[4] = new PropertyInfo("gg.client.protocol", info.getProperty("gg.client.protocol", "TCP"),
+        props[4] = new PropertyInfo("ignite.client.protocol", info.getProperty("ignite.client.protocol", "TCP"),
             "Communication protocol (TCP or HTTP).");
-        props[5] = new PropertyInfo("gg.client.connectTimeout", info.getProperty("gg.client.connectTimeout", "0"),
+        props[5] = new PropertyInfo("ignite.client.connectTimeout", info.getProperty("ignite.client.connectTimeout", "0"),
             "Socket connection timeout.");
-        props[6] = new PropertyInfo("gg.client.tcp.noDelay", info.getProperty("gg.client.tcp.noDelay", "true"),
+        props[6] = new PropertyInfo("ignite.client.tcp.noDelay", info.getProperty("ignite.client.tcp.noDelay", "true"),
             "Flag indicating whether TCP_NODELAY flag should be enabled for outgoing connections.");
-        props[7] = new PropertyInfo("gg.client.ssl.enabled", info.getProperty("gg.client.ssl.enabled", "false"),
+        props[7] = new PropertyInfo("ignite.client.ssl.enabled", info.getProperty("ignite.client.ssl.enabled", "false"),
             "Flag indicating that SSL is needed for connection.");
-        props[8] = new PropertyInfo("gg.client.ssl.protocol", info.getProperty("gg.client.ssl.protocol", "TLS"),
+        props[8] = new PropertyInfo("ignite.client.ssl.protocol", info.getProperty("ignite.client.ssl.protocol", "TLS"),
             "SSL protocol.");
-        props[9] = new PropertyInfo("gg.client.ssl.key.algorithm", info.getProperty("gg.client.ssl.key.algorithm",
+        props[9] = new PropertyInfo("ignite.client.ssl.key.algorithm", info.getProperty("ignite.client.ssl.key.algorithm",
             "SunX509"), "Key manager algorithm.");
-        props[10] = new PropertyInfo("gg.client.ssl.keystore.location",
-            info.getProperty("gg.client.ssl.keystore.location", ""),
+        props[10] = new PropertyInfo("ignite.client.ssl.keystore.location",
+            info.getProperty("ignite.client.ssl.keystore.location", ""),
             "Key store to be used by client to connect with Ignite topology.");
-        props[11] = new PropertyInfo("gg.client.ssl.keystore.password",
-            info.getProperty("gg.client.ssl.keystore.password", ""), "Key store password.");
-        props[12] = new PropertyInfo("gg.client.ssl.keystore.type", info.getProperty("gg.client.ssl.keystore.type",
+        props[11] = new PropertyInfo("ignite.client.ssl.keystore.password",
+            info.getProperty("ignite.client.ssl.keystore.password", ""), "Key store password.");
+        props[12] = new PropertyInfo("ignite.client.ssl.keystore.type", info.getProperty("ignite.client.ssl.keystore.type",
             "jks"), "Key store type.");
-        props[13] = new PropertyInfo("gg.client.ssl.truststore.location",
-            info.getProperty("gg.client.ssl.truststore.location", ""),
+        props[13] = new PropertyInfo("ignite.client.ssl.truststore.location",
+            info.getProperty("ignite.client.ssl.truststore.location", ""),
             "Trust store to be used by client to connect with Ignite topology.");
-        props[14] = new PropertyInfo("gg.client.ssl.keystore.password",
-            info.getProperty("gg.client.ssl.truststore.password", ""), "Trust store password.");
-        props[15] = new PropertyInfo("gg.client.ssl.truststore.type", info.getProperty("gg.client.ssl.truststore.type",
+        props[14] = new PropertyInfo("ignite.client.ssl.keystore.password",
+            info.getProperty("ignite.client.ssl.truststore.password", ""), "Trust store password.");
+        props[15] = new PropertyInfo("ignite.client.ssl.truststore.type", info.getProperty("ignite.client.ssl.truststore.type",
             "jks"), "Trust store type.");
-        props[16] = new PropertyInfo("gg.client.credentials", info.getProperty("gg.client.credentials", ""),
+        props[16] = new PropertyInfo("ignite.client.credentials", info.getProperty("ignite.client.credentials", ""),
             "Client credentials used in authentication process.");
-        props[17] = new PropertyInfo("gg.client.cache.top", info.getProperty("gg.client.cache.top", "false"),
+        props[17] = new PropertyInfo("ignite.client.cache.top", info.getProperty("ignite.client.cache.top", "false"),
             "Flag indicating that topology is cached internally. Cache will be refreshed in the background with " +
                 "interval defined by topologyRefreshFrequency property (see below).");
-        props[18] = new PropertyInfo("gg.client.topology.refresh", info.getProperty("gg.client.topology.refresh",
+        props[18] = new PropertyInfo("ignite.client.topology.refresh", info.getProperty("ignite.client.topology.refresh",
             "2000"), "Topology cache refresh frequency (ms).");
-        props[19] = new PropertyInfo("gg.client.idleTimeout", info.getProperty("gg.client.idleTimeout", "30000"),
+        props[19] = new PropertyInfo("ignite.client.idleTimeout", info.getProperty("ignite.client.idleTimeout", "30000"),
             "Maximum amount of time that connection can be idle before it is closed (ms).");
 
         return props;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/54f94922/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
index 3d01afa..ee16f94 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/GridClientConfiguration.java
@@ -160,7 +160,7 @@ public class GridClientConfiguration {
      * @throws GridClientException If parsing configuration failed.
      */
     public GridClientConfiguration(Properties in) throws GridClientException {
-        this("gg.client", in);
+        this("ignite.client", in);
     }
 
     /**


[04/50] incubator-ignite git commit: # ignite-791 schema-import profile.

Posted by sb...@apache.org.
# ignite-791 schema-import profile.


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

Branch: refs/heads/ignite-834
Commit: 58c55d4b676f930b186c32a1a188bd92debefb20
Parents: 60e240e
Author: Andrey <an...@gridgain.com>
Authored: Wed Apr 29 17:32:16 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Apr 29 17:32:16 2015 +0700

----------------------------------------------------------------------
 assembly/release-base.xml          |  2 ++
 assembly/release-schema-import.xml | 50 +++++++++++++++++++++++++++++++++
 pom.xml                            | 32 +++++++++++++++++++++
 3 files changed, 84 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/58c55d4b/assembly/release-base.xml
----------------------------------------------------------------------
diff --git a/assembly/release-base.xml b/assembly/release-base.xml
index 3297df8..6d6465e 100644
--- a/assembly/release-base.xml
+++ b/assembly/release-base.xml
@@ -73,6 +73,7 @@
                 <exclude>igniterouter.bat</exclude>
                 <exclude>**/target-classpath.bat</exclude>
                 <exclude>ignitevisorcmd.bat</exclude>
+                <exclude>ignite-schema-import.bat</exclude>
             </excludes>
         </fileSet>
 
@@ -89,6 +90,7 @@
                 <exclude>**/target-classpath.sh</exclude>
                 <exclude>**/service.sh</exclude>
                 <exclude>ignitevisorcmd.sh</exclude>
+                <exclude>ignite-schema-import.sh</exclude>
             </excludes>
         </fileSet>
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/58c55d4b/assembly/release-schema-import.xml
----------------------------------------------------------------------
diff --git a/assembly/release-schema-import.xml b/assembly/release-schema-import.xml
new file mode 100644
index 0000000..b746c83
--- /dev/null
+++ b/assembly/release-schema-import.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<assembly xmlns="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/plugins/maven-assembly-plugin/assembly/1.1.2
+          http://maven.apache.org/xsd/assembly-1.1.2.xsd">
+    <id>scala</id>
+
+    <includeBaseDirectory>false</includeBaseDirectory>
+
+    <formats>
+        <format>dir</format>
+    </formats>
+
+    <fileSets>
+        <fileSet>
+            <directory>bin</directory>
+            <outputDirectory>/</outputDirectory>
+            <includes>
+                <include>**/ignite-schema-import.bat</include>
+            </includes>
+        </fileSet>
+
+        <fileSet>
+            <directory>bin</directory>
+            <outputDirectory>/</outputDirectory>
+            <fileMode>0755</fileMode>
+            <includes>
+                <include>**/ignite-schema-import.sh</include>
+            </includes>
+        </fileSet>
+    </fileSets>
+</assembly>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/58c55d4b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b2cfa0f..eb73066 100644
--- a/pom.xml
+++ b/pom.xml
@@ -586,6 +586,22 @@
                                     <appendAssemblyId>false</appendAssemblyId>
                                 </configuration>
                             </execution>
+
+                            <execution>
+                                <id>schema-import-scripts</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>assembly/release-schema-import.xml</descriptor>
+                                    </descriptors>
+                                    <outputDirectory>target/release-package</outputDirectory>
+                                    <finalName>bin</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
                         </executions>
                     </plugin>
                 </plugins>
@@ -627,6 +643,22 @@
                                     <appendAssemblyId>false</appendAssemblyId>
                                 </configuration>
                             </execution>
+
+                            <execution>
+                                <id>schema-import-scripts</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>assembly/release-schema-import.xml</descriptor>
+                                    </descriptors>
+                                    <outputDirectory>target/release-package</outputDirectory>
+                                    <finalName>bin</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
                         </executions>
                     </plugin>
                 </plugins>


[30/50] incubator-ignite git commit: Merge branches 'ignite-752' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-752

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


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

Branch: refs/heads/ignite-834
Commit: f6ac27a35fe379b8024c824f00d3ce3f84d631fb
Parents: da85b11 ee5bae9
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 12:51:56 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 12:51:56 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 RELEASE_NOTES.txt                               |  13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++
 modules/cloud/pom.xml                           | 106 ++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 +++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 ++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../configuration/CacheConfiguration.java       | 259 +++++++--
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      | 346 +++++++++---
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/GridUpdateNotifier.java     |  66 ++-
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../apache/ignite/internal/IgniteKernal.java    |  83 ++-
 .../org/apache/ignite/internal/IgnitionEx.java  |  17 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |  36 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 +++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 +
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  54 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 ++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 +-
 .../cache/transactions/IgniteTxManager.java     | 117 +++-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 ++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../visor/cache/VisorCacheStartTask.java        | 155 ++++++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |  12 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  13 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 ++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 ------
 .../distributed/GridCacheLockAbstractTest.java  |  75 +++
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 +++++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 +++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 +++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 +++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 ++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 +++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 ++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 ++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 +++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 ++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 +++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 ++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 ++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 +++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 ++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 ++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 +++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 +++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 ++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  36 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 439 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 275 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 213 files changed, 9863 insertions(+), 3609 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6ac27a3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f6ac27a3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------


[38/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-4' into ignite-sprint-4


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

Branch: refs/heads/ignite-834
Commit: 0885ac0565c798a9f4a9b82897deb002d19ee47e
Parents: ea91d22 49a9923
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:10:53 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:10:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java      | 4 ----
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 ----
 .../ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java     | 8 ++++----
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 4 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------



[12/50] incubator-ignite git commit: Merge branches 'ignite-839' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-839

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


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

Branch: refs/heads/ignite-834
Commit: b01ff3824ba81b348da2d45342163c906e0f811c
Parents: e2ae597 d1a84a5
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 21:13:11 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 21:13:11 2015 +0700

----------------------------------------------------------------------
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++++
 modules/cloud/pom.xml                           | 108 +++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 ++++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 +++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../configuration/CacheConfiguration.java       |   4 +-
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../managers/communication/GridIoManager.java   |  22 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 +++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 +++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheStartTask.java        | 155 +++++++
 .../internal/visor/util/VisorTaskUtils.java     |  10 +
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++++
 modules/gce/pom.xml                             |  94 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 ++++++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 ++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 ++++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++--
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 +++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 ++++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 +++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 +++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 +++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 ++++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 +++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 ++++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 +++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 +++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 +++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 ++++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 +++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 +++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 ++++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 ++++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 +++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++--
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 +-
 pom.xml                                         |   2 +
 81 files changed, 5670 insertions(+), 352 deletions(-)
----------------------------------------------------------------------



[15/50] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-728' into ignite-728

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-728' into ignite-728


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

Branch: refs/heads/ignite-834
Commit: 8ae6b0319db2e92d7316533d9946bba51d2174c1
Parents: b58e1ac ef7d011
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:01:02 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:01:02 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |  44 ++++
 .../ignite/internal/GridJobContextImpl.java     |   7 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 165 +++++++++++++-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 ++++-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 ++--
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 ++++++++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 ++
 .../distributed/dht/GridDhtTxPrepareFuture.java |  10 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../distributed/near/GridNearLockRequest.java   |  54 ++---
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 ++++-
 .../cache/transactions/IgniteTxManager.java     | 117 ++++++++--
 .../ignite/internal/util/GridJavaProcess.java   |   4 +
 .../util/spring/IgniteSpringHelper.java         |  54 ++++-
 .../ignite/internal/visor/cache/VisorCache.java |  92 ++++----
 .../visor/node/VisorBasicConfiguration.java     |  11 +
 .../visor/node/VisorNodeDataCollectorJob.java   |   2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |  30 ++-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++++++++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 ++-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 ++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |   5 +-
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../util/spring/IgniteSpringHelperImpl.java     | 217 +++++++++++++++----
 .../IgniteStartFromStreamConfigurationTest.java |  50 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 .../config/VisorConfigurationCommand.scala      |   1 +
 .../scala/org/apache/ignite/visor/visor.scala   |  11 +
 pom.xml                                         |   5 +-
 40 files changed, 1396 insertions(+), 213 deletions(-)
----------------------------------------------------------------------



[49/50] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-834

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-4' into ignite-834


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

Branch: refs/heads/ignite-834
Commit: 086553ef14a75a0665f1e1346a100e858c1f39a5
Parents: 1255e58 54f9492
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue May 5 12:04:22 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue May 5 12:04:22 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   4 +-
 assembly/release-base.xml                       |   2 +
 assembly/release-schema-import.xml              |  50 +++
 .../streaming/wordcount/CacheConfig.java        |   5 -
 .../config/grid-client-config.properties        |  50 +--
 .../ClientPropertiesConfigurationSelfTest.java  |  12 +-
 modules/cloud/pom.xml                           |   4 +-
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../org/apache/ignite/IgniteJdbcDriver.java     |  81 +++--
 .../configuration/CacheConfiguration.java       | 259 +++++++++++---
 .../configuration/IgniteConfiguration.java      | 344 +++++++++++++++----
 .../ignite/internal/GridUpdateNotifier.java     |  66 +++-
 .../apache/ignite/internal/IgniteKernal.java    |  83 +++--
 .../org/apache/ignite/internal/IgnitionEx.java  |  15 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../managers/communication/GridIoManager.java   |   8 +-
 .../processors/cache/GridCacheTtlManager.java   | 168 +++++----
 .../processors/cache/GridCacheUtils.java        |   5 +-
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +--
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheStartTask.java        | 155 +++++++++
 .../internal/visor/util/VisorTaskUtils.java     |  10 +
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   4 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   4 -
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 -
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   8 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |  13 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   4 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 ---------
 .../tcp/TcpClientDiscoverySelfTest.java         |   8 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 modules/gce/pom.xml                             |   4 +-
 modules/schema-import/pom.xml                   |   6 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 +-
 pom.xml                                         | 230 +++++++++++--
 39 files changed, 1431 insertions(+), 585 deletions(-)
----------------------------------------------------------------------



[44/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the close method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the close method.


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

Branch: refs/heads/ignite-834
Commit: 601cf09930a16b1bf477d8407f5deed10c1f50c3
Parents: f298d72
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:32:56 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:32:56 2015 -0700

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteCache.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/601cf099/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index bf4080a..f9007a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -499,7 +499,7 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
         CacheEntryProcessor<K, V, T> entryProcessor, Object... args);
 
     /**
-     * Completely deletes the cache with all its data from the system.
+     * Completely deletes the cache with all its data from the system on all cluster nodes.
      */
     @Override void close();
 


[22/50] incubator-ignite git commit: # ignite-838

Posted by sb...@apache.org.
# ignite-838


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

Branch: refs/heads/ignite-834
Commit: dbd8d64ab256ba1a5f4104fe3eef8d5ba24bc2e6
Parents: ef9b2d9
Author: sboikov <se...@inria.fr>
Authored: Wed Apr 29 20:27:29 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Wed Apr 29 20:28:31 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/IgniteSpiAdapter.java  | 19 ++++++++++++++-----
 1 file changed, 14 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dbd8d64a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index b43f8a5..871512c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -67,7 +67,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
     private String name;
 
     /** Grid SPI context. */
-    private volatile IgniteSpiContext spiCtx = new GridDummySpiContext(null, false);
+    private volatile IgniteSpiContext spiCtx = new GridDummySpiContext(null, false, null);
 
     /** Discovery listener. */
     private GridLocalEventListener paramsLsnr;
@@ -190,7 +190,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         ClusterNode locNode = spiCtx == null ? null : spiCtx.localNode();
 
         // Set dummy no-op context.
-        spiCtx = new GridDummySpiContext(locNode, true);
+        spiCtx = new GridDummySpiContext(locNode, true, spiCtx);
     }
 
     /**
@@ -551,15 +551,24 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         /** */
         private final boolean stopping;
 
+        /** */
+        private final MessageFactory msgFactory;
+
+        /** */
+        private final MessageFormatter msgFormatter;
+
         /**
          * Create temp SPI context.
          *
          * @param locNode Local node.
          * @param stopping Node stopping flag.
+         * @param spiCtx SPI context.
          */
-        GridDummySpiContext(ClusterNode locNode, boolean stopping) {
+        GridDummySpiContext(ClusterNode locNode, boolean stopping, @Nullable IgniteSpiContext spiCtx) {
             this.locNode = locNode;
             this.stopping = stopping;
+            this.msgFactory = spiCtx != null ? spiCtx.messageFactory() : null;
+            this.msgFormatter = spiCtx != null ? spiCtx.messageFormatter() : null;
         }
 
         /** {@inheritDoc} */
@@ -711,12 +720,12 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
         /** {@inheritDoc} */
         @Override public MessageFormatter messageFormatter() {
-            return null;
+            return msgFormatter;
         }
 
         /** {@inheritDoc} */
         @Override public MessageFactory messageFactory() {
-            return null;
+            return msgFactory;
         }
 
         /** {@inheritDoc} */


[13/50] incubator-ignite git commit: "Version changed

Posted by sb...@apache.org.
"Version changed


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

Branch: refs/heads/ignite-834
Commit: 5854f1dbe312a5b8673bfdd93a2f3acfe699c202
Parents: d1a84a5
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Apr 29 17:27:47 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Apr 29 17:27:47 2015 +0300

----------------------------------------------------------------------
 modules/cloud/pom.xml                             | 4 +---
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/gce/pom.xml                               | 4 +---
 pom.xml                                           | 3 +--
 4 files changed, 4 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5854f1db/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 38c126f..8cb97d0 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -17,9 +17,7 @@
   limitations under the License.
 -->
 
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
     <modelVersion>4.0.0</modelVersion>
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5854f1db/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 549bde3..432b2ad 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.0.0
+ignite.version=1.0.3
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5854f1db/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index b12bb18..8b2a019 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -20,9 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0"
-         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
     <parent>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5854f1db/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 2a45704..0e84a50 100644
--- a/pom.xml
+++ b/pom.xml
@@ -20,8 +20,7 @@
 <!--
     POM file.
 -->
-<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-    xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
 <modelVersion>4.0.0</modelVersion>
 
     <parent>


[27/50] incubator-ignite git commit: minor

Posted by sb...@apache.org.
minor


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

Branch: refs/heads/ignite-834
Commit: 38dee89ac53b30c6543e6d6c9cf909649f73efba
Parents: 81ce0e6
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 11:38:42 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 11:38:42 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/communication/GridIoManager.java       | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/38dee89a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 6d2046f..16d582b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -35,6 +35,7 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.thread.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
@@ -186,7 +187,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
         mgmtPool = ctx.getManagementExecutorService();
         utilityCachePool = ctx.utilityCachePool();
         marshCachePool = ctx.marshallerCachePool();
-        affPool = Executors.newFixedThreadPool(1);
+        affPool = new IgniteThreadPoolExecutor(
+            "aff-" + ctx.gridName(),
+            1,
+            1,
+            0,
+            new LinkedBlockingQueue<Runnable>());
 
         getSpi().setListener(commLsnr = new CommunicationListener<Serializable>() {
             @Override public void onMessage(UUID nodeId, Serializable msg, IgniteRunnable msgC) {


[03/50] incubator-ignite git commit: # ignite-791 schema-import profile.

Posted by sb...@apache.org.
# ignite-791 schema-import profile.


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

Branch: refs/heads/ignite-834
Commit: 60e240e97ac57c9b11fa4a9affd574b2427af2ed
Parents: 0301207
Author: Andrey <an...@gridgain.com>
Authored: Wed Apr 29 16:16:54 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Apr 29 16:16:54 2015 +0700

----------------------------------------------------------------------
 modules/schema-import/pom.xml |  6 +++--
 pom.xml                       | 52 ++++++++++++++++++++++++++++++++++----
 2 files changed, 51 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/60e240e9/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 7c49cab..0684d11 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -84,9 +84,11 @@
 
     <profiles>
         <profile>
-            <id>jfxrt.jar</id>
+            <id>schema-import</id>
             <activation>
-                <jdk>[1.7,1.8)</jdk>
+                <file>
+                    <exists>${java.home}/lib/jfxrt.jar</exists>
+                </file>
             </activation>
             <dependencies>
                 <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/60e240e9/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5c232de..b2cfa0f 100644
--- a/pom.xml
+++ b/pom.xml
@@ -114,6 +114,7 @@
             <id>dev-libs</id>
             <activation>
                 <activeByDefault>true</activeByDefault>
+                <jdk>[1.7,)</jdk>
             </activation>
             <build>
                 <plugins>
@@ -167,6 +168,7 @@
             <id>release</id>
             <activation>
                 <activeByDefault>true</activeByDefault>
+                <jdk>[1.7,)</jdk>
             </activation>
             <build>
                 <plugins>
@@ -432,6 +434,7 @@
 
             <activation>
                 <activeByDefault>true</activeByDefault>
+                <jdk>[1.7,)</jdk>
             </activation>
 
             <modules>
@@ -550,13 +553,52 @@
         </profile>
 
         <profile>
+            <id>java8-schema-import</id>
+            <activation>
+                <file>
+                    <exists>${java.home}/lib/ext/jfxrt.jar</exists>
+                </file>
+            </activation>
+            <modules>
+                <module>modules/schema-import</module>
+            </modules>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <version>2.4</version>
+                        <inherited>false</inherited>
+
+                        <executions>
+                            <execution>
+                                <id>dependencies-schema-import</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>assembly/dependencies-schema-import.xml</descriptor>
+                                    </descriptors>
+                                    <outputDirectory>target/release-package/bin</outputDirectory>
+                                    <finalName>include</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+
+
+        <profile>
             <id>schema-import</id>
             <activation>
-                <jdk>[1.8)</jdk>
-                <property>
-                    <name>java.vendor</name>
-                    <value>Oracle Corporation</value>
-                </property>
+                <file>
+                    <exists>${java.home}/lib/jfxrt.jar</exists>
+                </file>
             </activation>
             <modules>
                 <module>modules/schema-import</module>


[40/50] incubator-ignite git commit: deploy to site enabled

Posted by sb...@apache.org.
deploy to site enabled


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

Branch: refs/heads/ignite-834
Commit: 04369fe6d02c9fb2d5cc24eb5b0f9c334bbf92a1
Parents: 0885ac0
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:47:19 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:47:19 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/04369fe6/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 181540b..98541ef 100644
--- a/pom.xml
+++ b/pom.xml
@@ -577,29 +577,29 @@
                         </executions>
                     </plugin>
 
-                    <!--<plugin>-->
-                        <!--<groupId>org.apache.maven.plugins</groupId>-->
-                        <!--<artifactId>maven-site-plugin</artifactId>-->
-                        <!--<version>3.4</version>-->
-                        <!--<dependencies>-->
-                            <!--<dependency>-->
-                                <!--<groupId>org.apache.maven.wagon</groupId>-->
-                                <!--<artifactId>wagon-ssh</artifactId>-->
-                                <!--<version>2.8</version>-->
-                            <!--</dependency>-->
-                        <!--</dependencies>-->
-                        <!--<executions>-->
-                            <!--<execution>-->
-                                <!--<goals>-->
-                                    <!--<goal>deploy</goal>-->
-                                <!--</goals>-->
-                                <!--<phase>deploy</phase>-->
-                                <!--<configuration>-->
-                                    <!--<inputDirectory>${basedir}/target/site</inputDirectory>-->
-                                <!--</configuration>-->
-                            <!--</execution>-->
-                        <!--</executions>-->
-                    <!--</plugin>-->
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-site-plugin</artifactId>
+                        <version>3.4</version>
+                        <dependencies>
+                            <dependency>
+                                <groupId>org.apache.maven.wagon</groupId>
+                                <artifactId>wagon-ssh</artifactId>
+                                <version>2.8</version>
+                            </dependency>
+                        </dependencies>
+                        <executions>
+                            <execution>
+                                <goals>
+                                    <goal>deploy</goal>
+                                </goals>
+                                <phase>deploy</phase>
+                                <configuration>
+                                    <inputDirectory>${basedir}/target/site</inputDirectory>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
                 </plugins>
             </build>
         </profile>


[18/50] incubator-ignite git commit: Merge branches 'ignite-791' and 'ignite-sprint-4' of http://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-791

Posted by sb...@apache.org.
Merge branches 'ignite-791' and 'ignite-sprint-4' of http://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-791


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

Branch: refs/heads/ignite-834
Commit: 51bf584e8d4d49b21dc5a022b8152eb9b6edca46
Parents: 58c55d4 47136b5
Author: anovikov <an...@gridgain.com>
Authored: Wed Apr 29 23:03:22 2015 +0700
Committer: anovikov <an...@gridgain.com>
Committed: Wed Apr 29 23:03:22 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  13 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |   2 +-
 .../clients/src/test/resources/spring-cache.xml |   4 +-
 .../src/test/resources/spring-server-node.xml   |   4 +-
 .../test/resources/spring-server-ssl-node.xml   |   4 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++
 modules/cloud/pom.xml                           | 106 ++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 +++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 ++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   2 +-
 .../main/java/org/apache/ignite/Ignition.java   |   2 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |   2 +-
 .../configuration/CacheConfiguration.java       |   4 +-
 .../configuration/ConnectorConfiguration.java   |   2 +-
 .../configuration/IgniteConfiguration.java      |   2 +-
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  16 +
 .../managers/communication/GridIoManager.java   |  28 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../deployment/GridDeploymentManager.java       |   2 +-
 .../GridDeploymentPerVersionStore.java          |   3 +-
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheMvccManager.java  |   2 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../cache/GridCacheSharedContext.java           |   2 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |  18 +-
 .../distributed/dht/GridDhtLockRequest.java     |  38 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  | 102 +++-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   6 +
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  23 +
 .../cache/distributed/dht/GridDhtTxRemote.java  |   7 +
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  54 +-
 .../cache/distributed/near/GridNearTxLocal.java |   6 +
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 ++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../cache/transactions/IgniteInternalTx.java    |   5 +
 .../cache/transactions/IgniteTxAdapter.java     |  10 +
 .../cache/transactions/IgniteTxHandler.java     |  57 +-
 .../cache/transactions/IgniteTxManager.java     | 117 +++-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 ++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../util/tostring/GridToStringBuilder.java      |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |   7 -
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +-
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheNodesTask.java        |  74 +++
 .../visor/cache/VisorCacheStartTask.java        | 155 ++++++
 .../cache/VisorCacheStoreConfiguration.java     |  35 --
 .../cache/VisorCacheTypeFieldMetadata.java      |  36 +-
 .../visor/cache/VisorCacheTypeMetadata.java     |  99 +---
 .../internal/visor/igfs/VisorIgfsMetrics.java   | 128 +----
 .../node/VisorNodeEventsCollectorTask.java      |  58 +-
 .../internal/visor/query/VisorQueryArg.java     |  19 +-
 .../internal/visor/query/VisorQueryCursor.java  |   1 -
 .../internal/visor/query/VisorQueryJob.java     |   9 +-
 .../internal/visor/query/VisorQueryTask.java    |  41 --
 .../internal/visor/util/VisorEventMapper.java   |  13 +
 .../internal/visor/util/VisorTaskUtils.java     |  12 +-
 .../apache/ignite/logger/java/JavaLogger.java   |  12 +-
 .../apache/ignite/marshaller/Marshaller.java    |  14 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |  10 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../apache/ignite/resources/LoggerResource.java |   2 +-
 .../apache/ignite/resources/SpringResource.java |   2 +-
 .../org/apache/ignite/services/Service.java     |   2 +-
 .../ignite/services/ServiceConfiguration.java   |   2 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |   2 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |   2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   4 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |  10 +-
 .../jobstealing/JobStealingCollisionSpi.java    |  14 +-
 .../PriorityQueueCollisionSpi.java              |   6 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  12 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../memory/MemoryEventStorageSpi.java           |  10 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |  10 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   6 +-
 .../spi/failover/never/NeverFailoverSpi.java    |   8 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   4 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |  12 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  10 +-
 .../WeightedRandomLoadBalancingSpi.java         |  10 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../src/test/config/load/merge-sort-base.xml    |   2 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java | 147 +++++
 .../cache/IgniteCachePutAllRestartTest.java     | 203 +++++++
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |  29 +-
 .../distributed/GridCacheLockAbstractTest.java  |  75 +++
 ...xOriginatingNodeFailureAbstractSelfTest.java |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   7 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |   2 -
 .../near/IgniteCacheNearTxRollbackTest.java     | 133 +++++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 .../logger/java/IgniteJavaLoggerTest.java       |  65 ---
 .../ignite/logger/java/JavaLoggerTest.java      |  65 +++
 .../junits/logger/GridTestLog4jLogger.java      |  10 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheFailoverTestSuite.java           |   8 +-
 .../testsuites/IgniteCacheRestartTestSuite.java |   3 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 .../testsuites/IgniteLoggingSelfTestSuite.java  |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 +++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 +++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/hadoop/HadoopProcessor.java      |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 +++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++-
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 ++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 +++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 ++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 ++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 +++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 ++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 +++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 ++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 ++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 ++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 +++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 ++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 ++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 ++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 +++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 +++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 ++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++-
 .../ignite/logger/jcl/IgniteJclLogger.java      | 167 ------
 .../org/apache/ignite/logger/jcl/JclLogger.java | 167 ++++++
 .../ignite/logger/jcl/IgniteJclLoggerTest.java  |  48 --
 .../apache/ignite/logger/jcl/JclLoggerTest.java |  48 ++
 .../ignite/testsuites/IgniteJclTestSuite.java   |   2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |   8 +-
 .../ignite/logger/slf4j/GridSlf4jLogger.java    | 138 -----
 .../apache/ignite/logger/slf4j/Slf4jLogger.java | 138 +++++
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |   2 +-
 .../ignite/visor/commands/VisorConsole.scala    | 314 ++++++-----
 .../visor/commands/VisorConsoleCommand.scala    |  77 ---
 .../ignite/visor/commands/VisorTextTable.scala  | 539 ------------------
 .../visor/commands/ack/VisorAckCommand.scala    |  42 +-
 .../commands/alert/VisorAlertCommand.scala      |  35 +-
 .../commands/cache/VisorCacheClearCommand.scala |  51 +-
 .../commands/cache/VisorCacheCommand.scala      |  36 +-
 .../commands/cache/VisorCacheScanCommand.scala  |  60 +-
 .../commands/cache/VisorCacheStopCommand.scala  |  30 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  66 +--
 .../commands/common/VisorConsoleCommand.scala   |  90 +++
 .../visor/commands/common/VisorTextTable.scala  | 543 +++++++++++++++++++
 .../config/VisorConfigurationCommand.scala      | 439 +++++++--------
 .../commands/deploy/VisorDeployCommand.scala    |  47 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |  58 +-
 .../commands/events/VisorEventsCommand.scala    | 338 +++++-------
 .../visor/commands/gc/VisorGcCommand.scala      | 130 ++---
 .../visor/commands/kill/VisorKillCommand.scala  |  53 +-
 .../visor/commands/node/VisorNodeCommand.scala  |  47 +-
 .../visor/commands/ping/VisorPingCommand.scala  |  41 +-
 .../commands/start/VisorStartCommand.scala      |  34 +-
 .../commands/tasks/VisorTasksCommand.scala      |  76 +--
 .../commands/top/VisorTopologyCommand.scala     |  36 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |  32 +-
 .../scala/org/apache/ignite/visor/visor.scala   | 275 +++++++---
 .../ignite/visor/VisorTextTableSpec.scala       |   3 +-
 pom.xml                                         |   5 +-
 205 files changed, 9098 insertions(+), 3257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/51bf584e/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index eb73066,0e84a50..03600e6
--- a/pom.xml
+++ b/pom.xml
@@@ -74,7 -73,10 +73,9 @@@
          <module>modules/log4j</module>
          <module>modules/slf4j</module>
          <module>modules/jcl</module>
 -        <module>modules/schema-import</module>
          <module>modules/codegen</module>
+         <module>modules/gce</module>
+         <module>modules/cloud</module>
      </modules>
  
      <profiles>


[08/50] incubator-ignite git commit: Merge branches 'ignite-789' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-789

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


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

Branch: refs/heads/ignite-834
Commit: be7bfda54b0e1f70fed377d94c6f2955ac742858
Parents: 96f3c03 2b02dd0
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 17:53:48 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 17:53:48 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  22 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++++
 modules/cloud/pom.xml                           | 108 +++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 ++++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 +++++
 .../processors/cache/IgniteCacheProxy.java      |  10 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../distributed/GridCacheLockAbstractTest.java  |  75 ++++
 .../DataStreamerMultiThreadedSelfTest.java      |   2 +
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++++
 modules/gce/pom.xml                             |  94 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 ++++++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 ++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 pom.xml                                         |   2 +
 22 files changed, 2049 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[17/50] incubator-ignite git commit: review

Posted by sb...@apache.org.
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/35e90b8a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/35e90b8a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/35e90b8a

Branch: refs/heads/ignite-834
Commit: 35e90b8aa7d90680625530701ba42a15d1a03808
Parents: d59e4c4
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Apr 29 18:17:16 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Apr 29 18:17:16 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheTtlManager.java  | 6 +++++-
 .../processors/cache/IgniteCacheEntryListenerAbstractTest.java | 2 +-
 2 files changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35e90b8a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index d8af2b9..546cf57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -73,6 +73,8 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
      * @param entry Entry to remove.
      */
     public void removeTrackedEntry(GridCacheMapEntry entry) {
+        assert Thread.holdsLock(entry);
+
         pendingEntries.remove(new EntryWrapper(entry));
     }
 
@@ -93,7 +95,9 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
 
         GridCacheVersion obsoleteVer = null;
 
-        int size = pendingEntries.sizex();
+        // Make sure that worker thread (e.g. sys pool) or user thread
+        // will not be trapped.
+        int size = Math.min(pendingEntries.sizex(), 1024);
 
         while (!sizeLimited || size-- > 0) {
             EntryWrapper e = pendingEntries.pollFirst();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/35e90b8a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index 544fe6c..bb449e0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -712,7 +712,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         expirePlcCache.put(key, 10);
 
-        U.sleep(500);
+        U.sleep(700);
 
         if (!eagerTtl())
             assertNull(primaryCache(key, cache.getName()).get(key)); // Provoke expire event if eager ttl is disabled.


[39/50] incubator-ignite git commit: #ignite-732: IgniteCache.size() should not fail in case of topology changes.

Posted by sb...@apache.org.
#ignite-732: IgniteCache.size() should not fail in case of topology changes.


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

Branch: refs/heads/ignite-834
Commit: 139aa270ae61494c0757867f2dc531ec7251b1da
Parents: 0885ac0
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Apr 30 18:43:56 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Apr 30 18:43:56 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeTaskAdapter.java      |  14 +-
 .../processors/cache/GridCacheAdapter.java      | 503 ++++++++++---------
 .../processors/cache/GridCacheProcessor.java    | 109 ++--
 3 files changed, 349 insertions(+), 277 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
index c2ad198..87081fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskAdapter.java
@@ -24,15 +24,16 @@ import java.util.*;
 
 /**
  * Convenience adapter for {@link ComputeTask} interface. Here is an example of
- * how {@code GridComputeTaskAdapter} can be used:
+ * how {@code ComputeTaskAdapter} can be used:
  * <pre name="code" class="java">
- * public class MyFooBarTask extends GridComputeTaskAdapter&lt;String, String&gt; {
+ * public class MyFooBarTask extends ComputeTaskAdapter&lt;String, String&gt; {
  *     // Inject load balancer.
  *     &#64;LoadBalancerResource
  *     ComputeLoadBalancer balancer;
  *
  *     // Map jobs to grid nodes.
- *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg) throws IgniteCheckedException {
+ *     public Map&lt;? extends ComputeJob, GridNode&gt; map(List&lt;GridNode&gt; subgrid, String arg)
+ *         throws IgniteCheckedException {
  *         Map&lt;MyFooBarJob, GridNode&gt; jobs = new HashMap&lt;MyFooBarJob, GridNode&gt;(subgrid.size());
  *
  *         // In more complex cases, you can actually do
@@ -76,8 +77,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * <p>
      * If remote job resulted in exception ({@link ComputeJobResult#getException()} is not {@code null}),
      * then {@link ComputeJobResultPolicy#FAILOVER} policy will be returned if the exception is instance
-     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException}, which means that
-     * remote node either failed or job execution was rejected before it got a chance to start. In all
+     * of {@link org.apache.ignite.cluster.ClusterTopologyException} or {@link ComputeExecutionRejectedException},
+     * which means that remote node either failed or job execution was rejected before it got a chance to start. In all
      * other cases the exception will be rethrown which will ultimately cause task to fail.
      *
      * @param res Received remote grid executable result.
@@ -87,7 +88,8 @@ public abstract class ComputeTaskAdapter<T, R> implements ComputeTask<T, R> {
      * @throws IgniteException If handling a job result caused an error effectively rejecting
      *      a failover. This exception will be thrown out of {@link ComputeTaskFuture#get()} method.
      */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
+        throws IgniteException {
         IgniteException e = res.getException();
 
         // Try to failover if result is failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 3f4e97b..39f19b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -75,6 +75,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** */
     private static final long serialVersionUID = 0L;
 
+    /** Failed result. */
+    private static final Object FAIL = new Integer(-1);
+
     /** clearLocally() split threshold. */
     public static final int CLEAR_ALL_SPLIT_THRESHOLD = 10000;
 
@@ -882,7 +885,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<Cache.Entry<K, V>> entrySet() {
-        return entrySet((CacheEntryPredicate[]) null);
+        return entrySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
@@ -897,17 +900,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[]) null);
+        return keySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
-        return primaryKeySet((CacheEntryPredicate[]) null);
+        return primaryKeySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<V> values() {
-        return values((CacheEntryPredicate[]) null);
+        return values((CacheEntryPredicate[])null);
     }
 
     /**
@@ -1080,36 +1083,31 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void clear() throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocally();
-
-        clearRemotes(0, new GlobalClearAllCallable(name()));
+        clearAll(0, new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
     }
 
     /** {@inheritDoc} */
     @Override public void clear(K key) throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocally(key);
-
-        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
+        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public void clearAll(Set<? extends K> keys) throws IgniteCheckedException {
-        // Clear local cache synchronously.
-        clearLocallyAll(keys);
-
-        clearRemotes(0, new GlobalClearKeySetCallable<K, V>(name(), keys));
+        clearAll(0, new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            keys));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(K key) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), Collections.singleton(key)));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            Collections.singleton(key)));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync(Set<? extends K> keys) {
-        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), keys));
+        return clearAsync(new GlobalClearKeySetCallable<K, V>(name(), ctx.affinity().affinityTopologyVersion(),
+            keys));
     }
 
     /**
@@ -1118,19 +1116,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param clearCall Global clear callable object.
      * @throws IgniteCheckedException In case of cache could not be cleared on any of the nodes.
      */
-    private void clearRemotes(long timeout, GlobalClearCallable clearCall) throws IgniteCheckedException {
+    private void clearAll(long timeout, TopologyVersionAwareCallable clearCall) throws IgniteCheckedException {
         try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes =
-                ctx.grid().cluster().forCacheNodes(name(), true, true, false).forRemotes().nodes();
-
             IgniteInternalFuture<Object> fut = null;
 
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
+            ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
 
-                fut = ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
-            }
+            fut = new ClearFuture(ctx, clearCall);
 
             if (fut != null)
                 fut.get();
@@ -1149,19 +1141,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> clearAsync() {
-        return clearAsync(new GlobalClearAllCallable(name()));
+        return clearAsync(new GlobalClearAllCallable(name(), ctx.affinity().affinityTopologyVersion()));
     }
 
     /**
      * @param clearCall Global clear callable object.
      * @return Future.
      */
-    private IgniteInternalFuture<?> clearAsync(GlobalClearCallable clearCall) {
+    private IgniteInternalFuture<?> clearAsync(TopologyVersionAwareCallable clearCall) {
         Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name(), true, true, false).nodes();
 
         if (!nodes.isEmpty()) {
-            IgniteInternalFuture<Object> fut =
-                ctx.closures().callAsyncNoFailover(BROADCAST, clearCall, nodes, true);
+            IgniteInternalFuture<Object> fut = new ClearFuture(ctx, clearCall);
 
             return fut.chain(new CX1<IgniteInternalFuture<Object>, Object>() {
                 @Override public Object applyx(IgniteInternalFuture<Object> fut) throws IgniteCheckedException {
@@ -2117,7 +2108,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 throws IgniteCheckedException {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys,
                     new C1<K, EntryProcessor<K, V, Object>>() {
-                            @Override public EntryProcessor apply(K k) {
+                        @Override public EntryProcessor apply(K k) {
                             return entryProcessor;
                         }
                     });
@@ -2145,7 +2136,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp() {
             @Override public IgniteInternalFuture<GridCacheReturn> inOp(IgniteTxLocalAdapter tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
-                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>) entryProcessor);
+                    Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
 
                 return tx.invokeAsync(ctx, invokeMap, args);
             }
@@ -2371,7 +2362,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noValArray())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>) RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
             }
 
             @Override public String toString() {
@@ -2526,7 +2517,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasValArray()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2915,7 +2906,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(oldVal);
 
-                return (GridCacheReturn) tx.putAllAsync(ctx,
+                return tx.putAllAsync(ctx,
                         F.t(key, newVal),
                         true,
                         null,
@@ -3017,7 +3008,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     ctx.deploy().registerClass(val);
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, false,
-                        ctx.equalsValArray(val)).get().success();
+                    ctx.equalsValArray(val)).get().success();
             }
 
             @Override public String toString() {
@@ -3230,10 +3221,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         TransactionConfiguration cfg = ctx.gridConfig().getTransactionConfiguration();
 
         return txStart(
-                concurrency,
-                isolation,
-                cfg.getDefaultTxTimeout(),
-                0
+            concurrency,
+            isolation,
+            cfg.getDefaultTxTimeout(),
+            0
         );
     }
 
@@ -3576,22 +3567,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (nodes.isEmpty())
             return new GridFinishedFuture<>(0);
 
-        IgniteInternalFuture<Collection<Integer>> fut =
-            ctx.closures().broadcastNoFailover(new SizeCallable(ctx.name(), peekModes), null, nodes);
-
-        return fut.chain(new CX1<IgniteInternalFuture<Collection<Integer>>, Integer>() {
-            @Override public Integer applyx(IgniteInternalFuture<Collection<Integer>> fut)
-            throws IgniteCheckedException {
-                Collection<Integer> res = fut.get();
-
-                int totalSize = 0;
-
-                for (Integer size : res)
-                    totalSize += size;
-
-                return totalSize;
-            }
-        });
+        return new SizeFuture(peekModes, ctx, modes.near);
     }
 
     /** {@inheritDoc} */
@@ -3675,7 +3651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return F.iterator(iterator(),
             new IgniteClosure<Cache.Entry<K, V>, Cache.Entry<K, V>>() {
                 private IgniteCacheExpiryPolicy expiryPlc =
-                        ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
+                    ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
 
                 @Override public Cache.Entry<K, V> apply(Cache.Entry<K, V> lazyEntry) {
                     CacheOperationContext prev = ctx.gate().enter(opCtx);
@@ -3909,50 +3885,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Gets cache global size (with or without backups).
-     *
-     * @param primaryOnly {@code True} if only primary sizes should be included.
-     * @return Global size.
-     * @throws IgniteCheckedException If internal task execution failed.
-     */
-    private int globalSize(boolean primaryOnly) throws IgniteCheckedException {
-        try {
-            // Send job to remote nodes only.
-            Collection<ClusterNode> nodes = ctx.grid().cluster().forCacheNodes(name()).forRemotes().nodes();
-
-            IgniteInternalFuture<Collection<Integer>> fut = null;
-
-            if (!nodes.isEmpty()) {
-                ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
-
-                fut = ctx.closures().broadcastNoFailover(new GlobalSizeCallable(name(), primaryOnly), null, nodes);
-            }
-
-            // Get local value.
-            int globalSize = primaryOnly ? primarySize() : size();
-
-            if (fut != null) {
-                for (Integer i : fut.get())
-                    globalSize += i;
-            }
-
-            return globalSize;
-        }
-        catch (ClusterGroupEmptyCheckedException ignore) {
-            if (log.isDebugEnabled())
-                log.debug("All remote nodes left while cache clearLocally [cacheName=" + name() + "]");
-
-            return primaryOnly ? primarySize() : size();
-        }
-        catch (ComputeTaskTimeoutCheckedException e) {
-            U.warn(log, "Timed out waiting for remote nodes to finish cache clear (consider increasing " +
-                "'networkTimeout' configuration property) [cacheName=" + name() + "]");
-
-            throw e;
-        }
-    }
-
-    /**
      * @param op Cache operation.
      * @param <T> Return type.
      * @return Operation result.
@@ -4893,47 +4825,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Internal callable which performs clear operation on a cache with the given name.
-     */
-    @GridInternal
-    private static abstract class GlobalClearCallable implements Callable<Object>, Externalizable {
-        /** Cache name. */
-        protected String cacheName;
-
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        protected Ignite ignite;
-
-        /**
-         * Empty constructor for serialization.
-         */
-        public GlobalClearCallable() {
-            // No-op.
-        }
-
-        /**
-         * @param cacheName Cache name.
-         */
-        protected GlobalClearCallable(String cacheName) {
-            this.cacheName = cacheName;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
-        }
-    }
-
-    /**
      * Global clear all.
      */
     @GridInternal
-    private static class GlobalClearAllCallable extends GlobalClearCallable {
+    private static class GlobalClearAllCallable extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4946,24 +4841,30 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          */
-        private GlobalClearAllCallable(String cacheName) {
-            super(cacheName);
+        private GlobalClearAllCallable(String cacheName, AffinityTopologyVersion topVer) {
+            super(cacheName, topVer);
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
+        @Override protected Object callLocal() {
             ((IgniteEx)ignite).cachex(cacheName).clearLocally();
 
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
+        }
     }
 
     /**
      * Global clear keys.
      */
     @GridInternal
-    private static class GlobalClearKeySetCallable<K, V> extends GlobalClearCallable {
+    private static class GlobalClearKeySetCallable<K, V> extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -4979,33 +4880,25 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          * @param keys Keys to clear.
          */
-        private GlobalClearKeySetCallable(String cacheName, Set<? extends K> keys) {
-            super(cacheName);
+        protected GlobalClearKeySetCallable(String cacheName, AffinityTopologyVersion topVer, Set<? extends K> keys) {
+            super(cacheName, topVer);
 
             this.keys = keys;
         }
 
         /** {@inheritDoc} */
-        @Override public Object call() throws Exception {
-            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
-
-            return null;
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            return ctx.grid().cluster().forCacheNodes(ctx.name(), true, true, false).nodes();
         }
 
         /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            super.writeExternal(out);
-
-            out.writeObject(keys);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            super.readExternal(in);
+        @Override protected Object callLocal() {
+            ((IgniteEx)ignite).<K, V>cachex(cacheName).clearLocallyAll(keys);
 
-            keys = (Set<K>) in.readObject();
+            return null;
         }
     }
 
@@ -5013,127 +4906,202 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Internal callable for global size calculation.
      */
     @GridInternal
-    private static class SizeCallable extends IgniteClosureX<Object, Integer> implements Externalizable {
+    private static class GlobalSizeCallable extends TopologyVersionAwareCallable {
         /** */
         private static final long serialVersionUID = 0L;
 
-        /** Cache name. */
-        private String cacheName;
-
         /** Peek modes. */
         private CachePeekMode[] peekModes;
 
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
+        /** Near enable. */
+        private boolean nearEnable;
 
         /**
          * Required by {@link Externalizable}.
          */
-        public SizeCallable() {
+        public GlobalSizeCallable() {
             // No-op.
         }
 
         /**
          * @param cacheName Cache name.
+         * @param topVer Affinity topology version.
          * @param peekModes Cache peek modes.
          */
-        private SizeCallable(String cacheName, CachePeekMode[] peekModes) {
-            this.cacheName = cacheName;
+        private GlobalSizeCallable(String cacheName, AffinityTopologyVersion topVer, CachePeekMode[] peekModes, boolean nearEnable) {
+            super(cacheName, topVer);
+
             this.peekModes = peekModes;
+            this.nearEnable = nearEnable;
         }
 
         /** {@inheritDoc} */
-        @Override public Integer applyx(Object o) throws IgniteCheckedException {
-            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
-
-            assert cache != null : cacheName;
+        @Override protected Object callLocal() {
+            try {
+                IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
-            return cache.localSize(peekModes);
+                return cache == null ? 0 : cache.localSize(peekModes);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings("ForLoopReplaceableByForEach")
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
+        @Override protected Collection<ClusterNode> nodes(GridCacheContext ctx) {
+            IgniteClusterEx cluster = ctx.grid().cluster();
 
-            out.writeInt(peekModes.length);
+            ClusterGroup grp = nearEnable ? cluster.forCacheNodes(ctx.name(), true, true, false) : cluster.forDataNodes(ctx.name());
 
-            for (int i = 0; i < peekModes.length; i++)
-                U.writeEnum(out, peekModes[i]);
+            return grp.nodes();
         }
 
         /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
+        public String toString() {
+            return S.toString(GlobalSizeCallable.class, this);
+        }
+    }
 
-            int len = in.readInt();
+    /**
+     * Cache size future.
+     */
+    private static class SizeFuture extends RetryFuture {
+        /** Size. */
+        private int size = 0;
 
-            peekModes = new CachePeekMode[len];
+        /**
+         * @param peekModes Peek modes.
+         */
+        public SizeFuture(CachePeekMode[] peekModes, GridCacheContext ctx, boolean near) {
+            super(ctx, new GlobalSizeCallable(ctx.name(), ctx.affinity().affinityTopologyVersion(), peekModes, near));
+        }
 
-            for (int i = 0; i < len; i++)
-                peekModes[i] = CachePeekMode.fromOrdinal(in.readByte());
+        /** {@inheritDoc} */
+        @Override protected void onInit() {
+            size = 0;
         }
 
         /** {@inheritDoc} */
-        public String toString() {
-            return S.toString(SizeCallable.class, this);
+        @Override protected void onLocal(Object localRes) {
+            size += (Integer)localRes;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void allDone() {
+            onDone(size);
         }
     }
 
     /**
-     * Internal callable which performs {@link IgniteInternalCache#size()} or {@link IgniteInternalCache#primarySize()}
-     * operation on a cache with the given name.
+     * Cache clear future.
      */
-    @GridInternal
-    private static class GlobalSizeCallable implements IgniteClosure<Object, Integer>, Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
+    private static class ClearFuture extends RetryFuture {
+        /**
+         */
+        public ClearFuture(GridCacheContext ctx, TopologyVersionAwareCallable clearCall) {
+            super(ctx, clearCall);
+        }
 
-        /** Cache name. */
-        private String cacheName;
+        /** {@inheritDoc} */
+        @Override protected void onInit() {
+           // No-op.
+        }
 
-        /** Primary only flag. */
-        private boolean primaryOnly;
+        /** {@inheritDoc} */
+        @Override protected void onLocal(Object localRes) {
+            // No-op.
+        }
 
-        /** Injected grid instance. */
-        @IgniteInstanceResource
-        private Ignite ignite;
+        /** {@inheritDoc} */
+        @Override protected void allDone() {
+            onDone();
+        }
+    }
+
+    /**
+     * Retry future.
+     */
+    protected static abstract class RetryFuture<T> extends GridFutureAdapter<T> {
+        /** Context. */
+        private final GridCacheContext ctx;
+
+        /** Callable. */
+        private final TopologyVersionAwareCallable call;
+
+        /** Max retries count before issuing an error. */
+        private volatile int retries = 32;
 
         /**
-         * Empty constructor for serialization.
          */
-        public GlobalSizeCallable() {
-            // No-op.
+        public RetryFuture(GridCacheContext ctx, TopologyVersionAwareCallable call) {
+            this.ctx = ctx;
+            this.call = call;
+
+            init();
         }
 
         /**
-         * @param cacheName Cache name.
-         * @param primaryOnly Primary only flag.
+         * Init.
          */
-        private GlobalSizeCallable(String cacheName, boolean primaryOnly) {
-            this.cacheName = cacheName;
-            this.primaryOnly = primaryOnly;
-        }
+        private void init() {
+            Collection<ClusterNode> nodes = call.nodes(ctx);
 
-        /** {@inheritDoc} */
-        @Override public Integer apply(Object o) {
-            IgniteInternalCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
+            call.topologyVersion(ctx.affinity().affinityTopologyVersion());
 
-            return primaryOnly ? cache.primarySize() : cache.size();
-        }
+            IgniteInternalFuture<Collection<Object>> fut = ctx.closures().callAsyncNoFailover(BROADCAST,
+                F.asSet((Callable<Object>)call), nodes, true);
 
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeString(out, cacheName);
-            out.writeBoolean(primaryOnly);
-        }
+            fut.listen(new IgniteInClosure<IgniteInternalFuture<Collection<Object>>>() {
+                @Override public void apply(IgniteInternalFuture<Collection<Object>> fut) {
+                    try {
+                        Collection<Object> res = fut.get();
 
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            cacheName = U.readString(in);
-            primaryOnly = in.readBoolean();
+                        onInit();
+
+                        for (Object locRes : res) {
+                            if (locRes == FAIL) {
+                                if (retries-- > 0)
+                                    init();
+                                else {
+                                    onDone(new ClusterTopologyException("Failed to wait topology."));
+
+                                    return;
+                                }
+                            }
+
+                            onLocal(locRes);
+                        }
+
+                        allDone();
+                    }
+                    catch (IgniteCheckedException e) {
+                        if (X.hasCause(e, ClusterTopologyException.class)) {
+                            if (retries-- > 0)
+                                init();
+                            else
+                                onDone(e);
+                        }
+                        else
+                            onDone(e);
+                    }
+                }
+            });
         }
+
+        /**
+         * Init reducer.
+         */
+        protected abstract void onInit();
+
+        /**
+         * @param localRes Add local result to global result.
+         */
+        protected abstract void onLocal(Object localRes);
+
+        /**
+         * On done.
+         */
+        protected abstract void allDone();
     }
 
     /**
@@ -5697,4 +5665,89 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             metrics.addPutAndGetTimeNanos(System.nanoTime() - start);
         }
     }
+
+    /**
+     * Delayed callable class.
+     */
+    protected static abstract class TopologyVersionAwareCallable<K, V> implements Serializable, Callable<Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Injected grid instance. */
+        @IgniteInstanceResource
+        protected Ignite ignite;
+
+        /** Affinity topology version. */
+        protected AffinityTopologyVersion topVer;
+
+        /** Cache name. */
+        protected String cacheName;
+
+        /**
+         * Empty constructor for serialization.
+         */
+        public TopologyVersionAwareCallable() {
+            // No-op.
+        }
+
+        /**
+         * @param topVer Affinity topology version.
+         */
+        public TopologyVersionAwareCallable(String cacheName, AffinityTopologyVersion topVer) {
+            this.cacheName = cacheName;
+            this.topVer = topVer;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            if (!compareTopologyVersions())
+                return FAIL;
+
+            Object res = callLocal();
+
+            if (!compareTopologyVersions())
+                return FAIL;
+            else
+                return res;
+        }
+
+        /**
+         * Call local.
+         *
+         * @return Local result.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected abstract Object callLocal() throws IgniteCheckedException;
+
+        /**
+         * @param ctx Grid cache context.
+         * @return Nodes to call.
+         */
+        protected abstract Collection<ClusterNode> nodes(GridCacheContext ctx);
+
+        /**
+         * Compare topology versions.
+         */
+        public boolean compareTopologyVersions() {
+            GridCacheProcessor cacheProc = ((IgniteKernal) ignite).context().cache();
+
+            GridCacheAdapter<K, V> cacheAdapter = cacheProc.internalCache(cacheName);
+
+            if (cacheAdapter == null)
+                return false;
+
+            final GridCacheContext<K, V> ctx = cacheAdapter.context();
+
+            AffinityTopologyVersion locTopVer = ctx.affinity().affinityTopologyVersion();
+
+            return locTopVer.compareTo(topVer) == 0;
+        }
+
+        /**
+         * @param topVer Affinity topology version.
+         */
+        public void topologyVersion(AffinityTopologyVersion topVer) {
+            this.topVer = topVer;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/139aa270/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 c0026ab..77fa104 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
@@ -124,6 +124,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Must use JDK marshaller since it is used by discovery to fire custom events. */
     private Marshaller marshaller = new JdkMarshaller();
 
+    /** Count down latch for caches. */
+    private CountDownLatch cacheStartedLatch = new CountDownLatch(1);
+
     /**
      * @param ctx Kernal context.
      */
@@ -657,87 +660,92 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        if (ctx.config().isDaemon())
-            return;
+        try {
+            if (ctx.config().isDaemon())
+                return;
 
-        ClusterNode locNode = ctx.discovery().localNode();
+            ClusterNode locNode = ctx.discovery().localNode();
 
-        // Init cache plugin managers.
-        final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
+            // Init cache plugin managers.
+            final Map<String, CachePluginManager> cache2PluginMgr = new HashMap<>();
 
-        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            CacheConfiguration locCcfg = desc.cacheConfiguration();
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                CacheConfiguration locCcfg = desc.cacheConfiguration();
 
-            CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
+                CachePluginManager pluginMgr = new CachePluginManager(ctx, locCcfg);
 
-            cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
-        }
+                cache2PluginMgr.put(locCcfg.getName(), pluginMgr);
+            }
 
-        if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
-            for (ClusterNode n : ctx.discovery().remoteNodes()) {
-                checkTransactionConfiguration(n);
+            if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
+                for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                    checkTransactionConfiguration(n);
 
-                DeploymentMode locDepMode = ctx.config().getDeploymentMode();
-                DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
+                    DeploymentMode locDepMode = ctx.config().getDeploymentMode();
+                    DeploymentMode rmtDepMode = n.attribute(IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE);
 
-                CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
-                    locDepMode, rmtDepMode, true);
+                    CU.checkAttributeMismatch(log, null, n.id(), "deploymentMode", "Deployment mode",
+                        locDepMode, rmtDepMode, true);
 
-                for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-                    CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
+                    for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                        CacheConfiguration rmtCfg = desc.remoteConfiguration(n.id());
 
-                    if (rmtCfg != null) {
-                        CacheConfiguration locCfg = desc.cacheConfiguration();
+                        if (rmtCfg != null) {
+                            CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                        checkCache(locCfg, rmtCfg, n);
+                            checkCache(locCfg, rmtCfg, n);
 
-                        // Check plugin cache configurations.
-                        CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
+                            // Check plugin cache configurations.
+                            CachePluginManager pluginMgr = cache2PluginMgr.get(locCfg.getName());
 
-                        assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                            assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                        pluginMgr.validateRemotes(rmtCfg, n);
+                            pluginMgr.validateRemotes(rmtCfg, n);
+                        }
                     }
                 }
             }
-        }
 
-        // Start dynamic caches received from collect discovery data.
-        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
-            boolean started = desc.onStart();
+            // Start dynamic caches received from collect discovery data.
+            for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+                boolean started = desc.onStart();
 
-            assert started : "Failed to change started flag for locally configured cache: " + desc;
+                assert started : "Failed to change started flag for locally configured cache: " + desc;
 
-            desc.clearRemoteConfigurations();
+                desc.clearRemoteConfigurations();
 
-            CacheConfiguration ccfg = desc.cacheConfiguration();
+                CacheConfiguration ccfg = desc.cacheConfiguration();
 
-            IgnitePredicate filter = ccfg.getNodeFilter();
+                IgnitePredicate filter = ccfg.getNodeFilter();
 
-            if (filter.apply(locNode)) {
-                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
+                if (filter.apply(locNode)) {
+                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-                CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
+                    CachePluginManager pluginMgr = cache2PluginMgr.get(ccfg.getName());
 
-                assert pluginMgr != null : " Map=" + cache2PluginMgr;
+                    assert pluginMgr != null : " Map=" + cache2PluginMgr;
 
-                GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
 
-                ctx.dynamicDeploymentId(desc.deploymentId());
+                    ctx.dynamicDeploymentId(desc.deploymentId());
 
-                sharedCtx.addCacheContext(ctx);
+                    sharedCtx.addCacheContext(ctx);
 
-                GridCacheAdapter cache = ctx.cache();
+                    GridCacheAdapter cache = ctx.cache();
 
-                String name = ccfg.getName();
+                    String name = ccfg.getName();
 
-                caches.put(maskNull(name), cache);
+                    caches.put(maskNull(name), cache);
 
-                startCache(cache);
+                    startCache(cache);
 
-                jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+                }
             }
         }
+        finally {
+            cacheStartedLatch.countDown();
+        }
 
         ctx.marshallerContext().onMarshallerCacheStarted(ctx);
 
@@ -835,6 +843,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStop(boolean cancel) {
+        cacheStartedLatch.countDown();
+
         if (ctx.config().isDaemon())
             return;
 
@@ -2686,6 +2696,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting internal cache adapter: " + name);
 
+        try {
+            cacheStartedLatch.await();
+        }
+        catch (InterruptedException e) {
+            throw new IgniteException("Failed to wait starting caches.");
+        }
+
         return (GridCacheAdapter<K, V>)caches.get(maskNull(name));
     }
 


[07/50] incubator-ignite git commit: Merge branches 'ignite-839' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-839

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


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

Branch: refs/heads/ignite-834
Commit: e2ae5979ec494e3d3f829ff6b8e0978d9b1caae8
Parents: db8333d 18b4c39
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 17:43:21 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 17:43:21 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               | 22 +++++++-------------
 .../processors/cache/IgniteCacheProxy.java      | 10 ++++-----
 .../DataStreamerMultiThreadedSelfTest.java      |  2 ++
 3 files changed, 15 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[19/50] incubator-ignite git commit: ignite-786: Implemented.

Posted by sb...@apache.org.
ignite-786: Implemented.


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

Branch: refs/heads/ignite-834
Commit: bba8c2fefaa68712c6993482f74c947bdde8aaa9
Parents: 47136b5
Author: Artem Shutak <as...@gridgain.com>
Authored: Wed Apr 29 19:07:47 2015 +0300
Committer: Artem Shutak <as...@gridgain.com>
Committed: Wed Apr 29 19:07:47 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       | 255 +++++++++++---
 .../configuration/IgniteConfiguration.java      | 344 +++++++++++++++----
 2 files changed, 479 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bba8c2fe/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 63da524..df6b2ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -417,11 +417,14 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets cache name.
      *
      * @param name Cache name. May be <tt>null</tt>, but may not be empty string.
+     * @return {@code this} for chaining.
      */
-    public void setName(String name) {
+    public CacheConfiguration setName(String name) {
         A.ensure(name == null || !name.isEmpty(), "Name cannot be null or empty.");
 
         this.name = name;
+
+        return this;
     }
 
     /**
@@ -439,9 +442,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets cache eviction policy.
      *
      * @param evictPlc Cache expiration policy.
+     * @return {@code this} for chaining.
      */
-    public void setEvictionPolicy(@Nullable EvictionPolicy evictPlc) {
+    public CacheConfiguration setEvictionPolicy(@Nullable EvictionPolicy evictPlc) {
         this.evictPlc = evictPlc;
+
+        return this;
     }
 
     /**
@@ -453,9 +459,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
 
     /**
      * @param nearCfg Near cache configuration.
+     * @return {@code this} for chaining.
      */
-    public void setNearConfiguration(NearCacheConfiguration<K, V> nearCfg) {
+    public CacheConfiguration setNearConfiguration(NearCacheConfiguration<K, V> nearCfg) {
         this.nearCfg = nearCfg;
+
+        return this;
     }
 
     /**
@@ -472,9 +481,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets write synchronization mode.
      *
      * @param writeSync Write synchronization mode.
+     * @return {@code this} for chaining.
      */
-    public void setWriteSynchronizationMode(CacheWriteSynchronizationMode writeSync) {
+    public CacheConfiguration setWriteSynchronizationMode(CacheWriteSynchronizationMode writeSync) {
         this.writeSync = writeSync;
+
+        return this;
     }
 
     /**
@@ -490,9 +502,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets filter which determines on what nodes the cache should be started.
      *
      * @param nodeFilter Predicate specifying on which nodes the cache should be started.
+     * @return {@code this} for chaining.
      */
-    public void setNodeFilter(IgnitePredicate<ClusterNode> nodeFilter) {
+    public CacheConfiguration setNodeFilter(IgnitePredicate<ClusterNode> nodeFilter) {
         this.nodeFilter = nodeFilter;
+
+        return this;
     }
 
     /**
@@ -519,9 +534,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * (or the rest of the nodes for replicated cache).
      *
      * @param evictSync {@code true} if synchronized, {@code false} if not.
+     * @return {@code this} for chaining.
      */
-    public void setEvictSynchronized(boolean evictSync) {
+    public CacheConfiguration setEvictSynchronized(boolean evictSync) {
         this.evictSync = evictSync;
+
+        return this;
     }
 
     /**
@@ -539,9 +557,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets eviction key buffer size.
      *
      * @param evictKeyBufSize Eviction key buffer size.
+     * @return {@code this} for chaining.
      */
-    public void setEvictSynchronizedKeyBufferSize(int evictKeyBufSize) {
+    public CacheConfiguration setEvictSynchronizedKeyBufferSize(int evictKeyBufSize) {
         this.evictKeyBufSize = evictKeyBufSize;
+
+        return this;
     }
 
     /**
@@ -570,9 +591,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets concurrency level for synchronized evictions.
      *
      * @param evictSyncConcurrencyLvl Concurrency level for synchronized evictions.
+     * @return {@code this} for chaining.
      */
-    public void setEvictSynchronizedConcurrencyLevel(int evictSyncConcurrencyLvl) {
+    public CacheConfiguration setEvictSynchronizedConcurrencyLevel(int evictSyncConcurrencyLvl) {
         this.evictSyncConcurrencyLvl = evictSyncConcurrencyLvl;
+
+        return this;
     }
 
     /**
@@ -593,9 +617,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets timeout for synchronized evictions.
      *
      * @param evictSyncTimeout Timeout for synchronized evictions.
+     * @return {@code this} for chaining.
      */
-    public void setEvictSynchronizedTimeout(long evictSyncTimeout) {
+    public CacheConfiguration setEvictSynchronizedTimeout(long evictSyncTimeout) {
         this.evictSyncTimeout = evictSyncTimeout;
+
+        return this;
     }
 
     /**
@@ -622,9 +649,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets maximum eviction overflow ratio.
      *
      * @param evictMaxOverflowRatio Maximum eviction overflow ratio.
+     * @return {@code this} for chaining.
      */
-    public void setEvictMaxOverflowRatio(float evictMaxOverflowRatio) {
+    public CacheConfiguration setEvictMaxOverflowRatio(float evictMaxOverflowRatio) {
         this.evictMaxOverflowRatio = evictMaxOverflowRatio;
+
+        return this;
     }
 
     /**
@@ -648,9 +678,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets eviction filter.
      *
      * @param evictFilter Eviction filter.
+     * @return {@code this} for chaining.
      */
-    public void setEvictionFilter(EvictionFilter<K, V> evictFilter) {
+    public CacheConfiguration setEvictionFilter(EvictionFilter<K, V> evictFilter) {
         this.evictFilter = evictFilter;
+
+        return this;
     }
 
     /**
@@ -674,9 +707,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param eagerTtl {@code True} if Ignite should eagerly remove expired cache entries.
      * @see #isEagerTtl()
+     * @return {@code this} for chaining.
      */
-    public void setEagerTtl(boolean eagerTtl) {
+    public CacheConfiguration setEagerTtl(boolean eagerTtl) {
         this.eagerTtl = eagerTtl;
+
+        return this;
     }
 
     /**
@@ -693,9 +729,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Initial size for internal hash map.
      *
      * @param startSize Cache start size.
+     * @return {@code this} for chaining.
      */
-    public void setStartSize(int startSize) {
+    public CacheConfiguration setStartSize(int startSize) {
         this.startSize = startSize;
+
+        return this;
     }
 
     /**
@@ -734,9 +773,13 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * When not set, default value is {@link #DFLT_LOAD_PREV_VAL}.
      *
      * @param loadPrevVal Load previous value flag.
+     * @return {@code this} for chaining.
+     * @return {@code this} for chaining.
      */
-    public void setLoadPreviousValue(boolean loadPrevVal) {
+    public CacheConfiguration setLoadPreviousValue(boolean loadPrevVal) {
         this.loadPrevVal = loadPrevVal;
+
+        return this;
     }
 
     /**
@@ -755,8 +798,10 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * @param storeFactory Cache store factory.
      */
     @SuppressWarnings("unchecked")
-    public void setCacheStoreFactory(Factory<? extends CacheStore<? super K, ? super V>> storeFactory) {
+    public CacheConfiguration setCacheStoreFactory(Factory<? extends CacheStore<? super K, ? super V>> storeFactory) {
         this.storeFactory = storeFactory;
+
+        return this;
     }
 
     /**
@@ -772,9 +817,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets affinity for cache keys.
      *
      * @param aff Cache key affinity.
+     * @return {@code this} for chaining.
      */
-    public void setAffinity(AffinityFunction aff) {
+    public CacheConfiguration setAffinity(AffinityFunction aff) {
         this.aff = aff;
+
+        return this;
     }
 
     /**
@@ -792,9 +840,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets caching mode.
      *
      * @param cacheMode Caching mode.
+     * @return {@code this} for chaining.
      */
-    public void setCacheMode(CacheMode cacheMode) {
+    public CacheConfiguration setCacheMode(CacheMode cacheMode) {
         this.cacheMode = cacheMode;
+
+        return this;
     }
 
     /**
@@ -812,9 +863,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets cache atomicity mode.
      *
      * @param atomicityMode Cache atomicity mode.
+     * @return {@code this} for chaining.
      */
-    public void setAtomicityMode(CacheAtomicityMode atomicityMode) {
+    public CacheConfiguration setAtomicityMode(CacheAtomicityMode atomicityMode) {
         this.atomicityMode = atomicityMode;
+
+        return this;
     }
 
     /**
@@ -832,9 +886,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * cache (for other atomicity modes it will be ignored).
      *
      * @param atomicWriteOrderMode Cache write ordering mode.
+     * @return {@code this} for chaining.
      */
-    public void setAtomicWriteOrderMode(CacheAtomicWriteOrderMode atomicWriteOrderMode) {
+    public CacheConfiguration setAtomicWriteOrderMode(CacheAtomicWriteOrderMode atomicWriteOrderMode) {
         this.atomicWriteOrderMode = atomicWriteOrderMode;
+
+        return this;
     }
 
     /**
@@ -854,9 +911,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * If not set, default value is {@link #DFLT_BACKUPS}.
      *
      * @param backups Number of backup nodes for one partition.
+     * @return {@code this} for chaining.
      */
-    public void setBackups(int backups) {
+    public CacheConfiguration setBackups(int backups) {
         this.backups = backups;
+
+        return this;
     }
 
     /**
@@ -873,9 +933,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets default lock timeout in milliseconds. By default this value is defined by {@link #DFLT_LOCK_TIMEOUT}.
      *
      * @param dfltLockTimeout Default lock timeout.
+     * @return {@code this} for chaining.
      */
-    public void setDefaultLockTimeout(long dfltLockTimeout) {
+    public CacheConfiguration setDefaultLockTimeout(long dfltLockTimeout) {
         this.dfltLockTimeout = dfltLockTimeout;
+
+        return this;
     }
 
     /**
@@ -891,9 +954,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets invalidation flag for near cache entries in this transaction. Default is {@code false}.
      *
      * @param invalidate Flag to set this cache into invalidation-based mode. Default value is {@code false}.
+     * @return {@code this} for chaining.
      */
-    public void setInvalidate(boolean invalidate) {
+    public CacheConfiguration setInvalidate(boolean invalidate) {
         this.invalidate = invalidate;
+
+        return this;
     }
 
     /**
@@ -910,18 +976,24 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param tmLookupClsName Name of class implementing GridCacheTmLookup interface that is used to
      *      receive JTA transaction manager.
+     * @return {@code this} for chaining.
      */
-    public void setTransactionManagerLookupClassName(String tmLookupClsName) {
+    public CacheConfiguration setTransactionManagerLookupClassName(String tmLookupClsName) {
         this.tmLookupClsName = tmLookupClsName;
+
+        return this;
     }
 
     /**
      * Sets cache rebalance mode.
      *
      * @param rebalanceMode Rebalance mode.
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceMode(CacheRebalanceMode rebalanceMode) {
+    public CacheConfiguration setRebalanceMode(CacheRebalanceMode rebalanceMode) {
         this.rebalanceMode = rebalanceMode;
+
+        return this;
     }
 
     /**
@@ -959,9 +1031,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param rebalanceOrder Cache rebalance order.
      * @see #getRebalanceOrder()
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceOrder(int rebalanceOrder) {
+    public CacheConfiguration setRebalanceOrder(int rebalanceOrder) {
         this.rebalanceOrder = rebalanceOrder;
+
+        return this;
     }
 
     /**
@@ -980,9 +1055,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets rebalance batch size.
      *
      * @param rebalanceBatchSize Rebalance batch size.
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceBatchSize(int rebalanceBatchSize) {
+    public CacheConfiguration setRebalanceBatchSize(int rebalanceBatchSize) {
         this.rebalanceBatchSize = rebalanceBatchSize;
+
+        return this;
     }
 
     /**
@@ -999,9 +1077,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Flag indicating whether swap storage is enabled or not.
      *
      * @param swapEnabled {@code True} if swap storage is enabled.
+     * @return {@code this} for chaining.
      */
-    public void setSwapEnabled(boolean swapEnabled) {
+    public CacheConfiguration setSwapEnabled(boolean swapEnabled) {
         this.swapEnabled = swapEnabled;
+
+        return this;
     }
 
     /**
@@ -1025,9 +1106,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param maxConcurrentAsyncOps Maximum number of concurrent asynchronous operations.
      * @see #getMaxConcurrentAsyncOperations()
+     * @return {@code this} for chaining.
      */
-    public void setMaxConcurrentAsyncOperations(int maxConcurrentAsyncOps) {
+    public CacheConfiguration setMaxConcurrentAsyncOperations(int maxConcurrentAsyncOps) {
         this.maxConcurrentAsyncOps = maxConcurrentAsyncOps;
+
+        return this;
     }
 
     /**
@@ -1045,9 +1129,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets flag indicating whether write-behind is enabled.
      *
      * @param writeBehindEnabled {@code true} if write-behind is enabled.
+     * @return {@code this} for chaining.
      */
-    public void setWriteBehindEnabled(boolean writeBehindEnabled) {
+    public CacheConfiguration setWriteBehindEnabled(boolean writeBehindEnabled) {
         this.writeBehindEnabled = writeBehindEnabled;
+
+        return this;
     }
 
     /**
@@ -1070,9 +1157,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param writeBehindFlushSize Write-behind cache flush size.
      * @see #getWriteBehindFlushSize()
+     * @return {@code this} for chaining.
      */
-    public void setWriteBehindFlushSize(int writeBehindFlushSize) {
+    public CacheConfiguration setWriteBehindFlushSize(int writeBehindFlushSize) {
         this.writeBehindFlushSize = writeBehindFlushSize;
+
+        return this;
     }
 
     /**
@@ -1096,9 +1186,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param writeBehindFlushFreq Write-behind flush frequency in milliseconds.
      * @see #getWriteBehindFlushFrequency()
+     * @return {@code this} for chaining.
      */
-    public void setWriteBehindFlushFrequency(long writeBehindFlushFreq) {
+    public CacheConfiguration setWriteBehindFlushFrequency(long writeBehindFlushFreq) {
         this.writeBehindFlushFreq = writeBehindFlushFreq;
+
+        return this;
     }
 
     /**
@@ -1120,9 +1213,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param writeBehindFlushThreadCnt Count of flush threads.
      * @see #getWriteBehindFlushThreadCount()
+     * @return {@code this} for chaining.
      */
-    public void setWriteBehindFlushThreadCount(int writeBehindFlushThreadCnt) {
+    public CacheConfiguration setWriteBehindFlushThreadCount(int writeBehindFlushThreadCnt) {
         this.writeBehindFlushThreadCnt = writeBehindFlushThreadCnt;
+
+        return this;
     }
 
     /**
@@ -1144,9 +1240,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param writeBehindBatchSize Maximum batch size.
      * @see #getWriteBehindBatchSize()
+     * @return {@code this} for chaining.
      */
-    public void setWriteBehindBatchSize(int writeBehindBatchSize) {
+    public CacheConfiguration setWriteBehindBatchSize(int writeBehindBatchSize) {
         this.writeBehindBatchSize = writeBehindBatchSize;
+
+        return this;
     }
 
     /**
@@ -1166,9 +1265,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * for rebalancing than specified here (but never less threads).
      *
      * @param rebalancePoolSize Size of rebalancing thread pool.
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceThreadPoolSize(int rebalancePoolSize) {
+    public CacheConfiguration setRebalanceThreadPoolSize(int rebalancePoolSize) {
         this.rebalancePoolSize = rebalancePoolSize;
+
+        return this;
     }
 
     /**
@@ -1186,9 +1288,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets rebalance timeout (ms).
      *
      * @param rebalanceTimeout Rebalance timeout (ms).
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceTimeout(long rebalanceTimeout) {
+    public CacheConfiguration setRebalanceTimeout(long rebalanceTimeout) {
         this.rebalanceTimeout = rebalanceTimeout;
+
+        return this;
     }
 
     /**
@@ -1221,9 +1326,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets rebalance delay (see {@link #getRebalanceDelay()} for more information).
      *
      * @param rebalanceDelay Rebalance delay to set.
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceDelay(long rebalanceDelay) {
+    public CacheConfiguration setRebalanceDelay(long rebalanceDelay) {
         this.rebalanceDelay = rebalanceDelay;
+
+        return this;
     }
 
     /**
@@ -1254,9 +1362,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param rebalanceThrottle Time in milliseconds to wait between rebalance messages to avoid overloading of CPU,
      * {@code 0} to disable throttling.
+     * @return {@code this} for chaining.
      */
-    public void setRebalanceThrottle(long rebalanceThrottle) {
+    public CacheConfiguration setRebalanceThrottle(long rebalanceThrottle) {
         this.rebalanceThrottle = rebalanceThrottle;
+
+        return this;
     }
 
     /**
@@ -1278,9 +1389,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * described in {@link AffinityKeyMapper} documentation.
      *
      * @param affMapper Affinity mapper.
+     * @return {@code this} for chaining.
      */
-    public void setAffinityMapper(AffinityKeyMapper affMapper) {
+    public CacheConfiguration setAffinityMapper(AffinityKeyMapper affMapper) {
         this.affMapper = affMapper;
+
+        return this;
     }
 
     /**
@@ -1322,9 +1436,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * off-heap space without affecting JVM performance at all.
      *
      * @param offHeapMaxMem Maximum memory in bytes available to off-heap memory space.
+     * @return {@code this} for chaining.
      */
-    public void setOffHeapMaxMemory(long offHeapMaxMem) {
+    public CacheConfiguration setOffHeapMaxMemory(long offHeapMaxMem) {
         this.offHeapMaxMem = offHeapMaxMem;
+
+        return this;
     }
 
     /**
@@ -1343,9 +1460,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets memory mode for cache.
      *
      * @param memMode Memory mode.
+     * @return {@code this} for chaining.
      */
-    public void setMemoryMode(CacheMemoryMode memMode) {
+    public CacheConfiguration setMemoryMode(CacheMemoryMode memMode) {
         this.memMode = memMode;
+
+        return this;
     }
 
     /**
@@ -1362,9 +1482,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets cache interceptor.
      *
      * @param interceptor Cache interceptor.
+     * @return {@code this} for chaining.
      */
-    public void setInterceptor(CacheInterceptor<K, V> interceptor) {
+    public CacheConfiguration setInterceptor(CacheInterceptor<K, V> interceptor) {
         this.interceptor = interceptor;
+
+        return this;
     }
 
     /**
@@ -1380,9 +1503,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets collection of type metadata objects.
      *
      * @param typeMeta Collection of type metadata.
+     * @return {@code this} for chaining.
      */
-    public void setTypeMetadata(Collection<CacheTypeMetadata> typeMeta) {
+    public CacheConfiguration setTypeMetadata(Collection<CacheTypeMetadata> typeMeta) {
         this.typeMeta = typeMeta;
+
+        return this;
     }
 
     /**
@@ -1402,9 +1528,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets read from backup flag.
      *
      * @param readFromBackup {@code true} to allow reads from backups.
+     * @return {@code this} for chaining.
      */
-    public void setReadFromBackup(boolean readFromBackup) {
+    public CacheConfiguration setReadFromBackup(boolean readFromBackup) {
         this.readFromBackup = readFromBackup;
+
+        return this;
     }
 
     /**
@@ -1423,9 +1552,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param cpOnRead Copy on get flag.
      * @see #isCopyOnRead
+     * @return {@code this} for chaining.
      */
-    public void setCopyOnRead(boolean cpOnRead) {
+    public CacheConfiguration setCopyOnRead(boolean cpOnRead) {
         this.cpOnRead = cpOnRead;
+
+        return this;
     }
 
     /**
@@ -1433,9 +1565,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * to be used as user-defined functions from SQL queries.
      *
      * @param cls One or more classes with SQL functions.
+     * @return {@code this} for chaining.
      */
-    public void setSqlFunctionClasses(Class<?>... cls) {
+    public CacheConfiguration setSqlFunctionClasses(Class<?>... cls) {
         this.sqlFuncCls = cls;
+
+        return this;
     }
 
     /**
@@ -1461,9 +1596,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Gets timeout in milliseconds after which long query warning will be printed.
      *
      * @param longQryWarnTimeout Timeout in milliseconds.
+     * @return {@code this} for chaining.
      */
-    public void setLongQueryWarningTimeout(long longQryWarnTimeout) {
+    public CacheConfiguration setLongQueryWarningTimeout(long longQryWarnTimeout) {
         this.longQryWarnTimeout = longQryWarnTimeout;
+
+        return this;
     }
 
     /**
@@ -1483,9 +1621,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * also allows having special characters in table and field names.
      *
      * @param sqlEscapeAll Flag value.
+     * @return {@code this} for chaining.
      */
-    public void setSqlEscapeAll(boolean sqlEscapeAll) {
+    public CacheConfiguration setSqlEscapeAll(boolean sqlEscapeAll) {
         this.sqlEscapeAll = sqlEscapeAll;
+
+        return this;
     }
 
     /**
@@ -1517,12 +1658,15 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * from package {@link org.apache.ignite.cache.query.annotations}.
      *
      * @param indexedTypes Key and value type pairs.
+     * @return {@code this} for chaining.
      */
-    public void setIndexedTypes(Class<?>... indexedTypes) {
+    public CacheConfiguration setIndexedTypes(Class<?>... indexedTypes) {
         A.ensure(indexedTypes == null || (indexedTypes.length & 1) == 0,
             "Number of indexed types is expected to be even. Refer to method javadoc for details.");
 
         this.indexedTypes = indexedTypes;
+
+        return this;
     }
 
     /**
@@ -1542,9 +1686,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      *
      * @param size Cache size.
      * @see #setOffHeapMaxMemory(long)
+     * @return {@code this} for chaining.
      */
-    public void setSqlOnheapRowCacheSize(int size) {
+    public CacheConfiguration setSqlOnheapRowCacheSize(int size) {
         this.sqlOnheapRowCacheSize = size;
+
+        return this;
     }
 
     /**
@@ -1560,9 +1707,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
      * Sets cache plugin configurations.
      *
      * @param pluginCfgs Cache plugin configurations.
+     * @return {@code this} for chaining.
      */
-    public void setPluginConfigurations(CachePluginConfiguration... pluginCfgs) {
+    public CacheConfiguration setPluginConfigurations(CachePluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
+
+        return this;
     }
 
     /**
@@ -1576,9 +1726,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /**
      * Sets topology validator.
      * @param topValidator validator.
+     * @return {@code this} for chaining.
      */
-    public void setTopologyValidator(TopologyValidator topValidator) {
+    public CacheConfiguration setTopologyValidator(TopologyValidator topValidator) {
         this.topValidator = topValidator;
+
+        return this;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bba8c2fe/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 76f3043..ebe2b8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -533,9 +533,12 @@ public class IgniteConfiguration {
      * or in-memory data grid storage.
      *
      * @param daemon Daemon flag.
+     * @return {@code this} for chaining.
      */
-    public void setDaemon(boolean daemon) {
+    public IgniteConfiguration setDaemon(boolean daemon) {
         this.daemon = daemon;
+
+        return this;
     }
 
     /**
@@ -543,9 +546,12 @@ public class IgniteConfiguration {
      *
      * @param gridName Grid name to set. Can be {@code null}, which is default
      *      grid name.
+     * @return {@code this} for chaining.
      */
-    public void setGridName(String gridName) {
+    public IgniteConfiguration setGridName(String gridName) {
         this.gridName = gridName;
+
+        return this;
     }
 
     /**
@@ -577,9 +583,12 @@ public class IgniteConfiguration {
      *
      * @param userAttrs User attributes for this node.
      * @see IgniteConfiguration#getUserAttributes()
+     * @return {@code this} for chaining.
      */
-    public void setUserAttributes(Map<String, ?> userAttrs) {
+    public IgniteConfiguration setUserAttributes(Map<String, ?> userAttrs) {
         this.userAttrs = userAttrs;
+
+        return this;
     }
 
     /**
@@ -598,9 +607,12 @@ public class IgniteConfiguration {
      *
      * @param log Logger to use within grid.
      * @see IgniteConfiguration#getGridLogger()
+     * @return {@code this} for chaining.
      */
-    public void setGridLogger(IgniteLogger log) {
+    public IgniteConfiguration setGridLogger(IgniteLogger log) {
         this.log = log;
+
+        return this;
     }
 
     /**
@@ -715,9 +727,12 @@ public class IgniteConfiguration {
      *
      * @param poolSize Thread pool size to use within grid.
      * @see IgniteConfiguration#getPublicThreadPoolSize()
+     * @return {@code this} for chaining.
      */
-    public void setPublicThreadPoolSize(int poolSize) {
+    public IgniteConfiguration setPublicThreadPoolSize(int poolSize) {
         pubPoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -725,9 +740,12 @@ public class IgniteConfiguration {
      *
      * @param poolSize Thread pool size to use within grid.
      * @see IgniteConfiguration#getSystemThreadPoolSize()
+     * @return {@code this} for chaining.
      */
-    public void setSystemThreadPoolSize(int poolSize) {
+    public IgniteConfiguration setSystemThreadPoolSize(int poolSize) {
         sysPoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -735,9 +753,12 @@ public class IgniteConfiguration {
      *
      * @param poolSize Thread pool size to use within grid.
      * @see IgniteConfiguration#getManagementThreadPoolSize()
+     * @return {@code this} for chaining.
      */
-    public void setManagementThreadPoolSize(int poolSize) {
+    public IgniteConfiguration setManagementThreadPoolSize(int poolSize) {
         mgmtPoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -745,9 +766,12 @@ public class IgniteConfiguration {
      *
      * @param poolSize Thread pool size to use within grid.
      * @see IgniteConfiguration#getPeerClassLoadingThreadPoolSize()
+     * @return {@code this} for chaining.
      */
-    public void setPeerClassLoadingThreadPoolSize(int poolSize) {
+    public IgniteConfiguration setPeerClassLoadingThreadPoolSize(int poolSize) {
         p2pPoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -755,9 +779,12 @@ public class IgniteConfiguration {
      *
      * @param poolSize Executor service to use for outgoing IGFS messages.
      * @see IgniteConfiguration#getIgfsThreadPoolSize()
+     * @return {@code this} for chaining.
      */
-    public void setIgfsThreadPoolSize(int poolSize) {
+    public IgniteConfiguration setIgfsThreadPoolSize(int poolSize) {
         igfsPoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -766,9 +793,12 @@ public class IgniteConfiguration {
      * @param poolSize Default executor service size to use for utility cache messages.
      * @see IgniteConfiguration#getUtilityCacheThreadPoolSize()
      * @see IgniteConfiguration#getUtilityCacheKeepAliveTime()
+     * @return {@code this} for chaining.
      */
-    public void setUtilityCachePoolSize(int poolSize) {
+    public IgniteConfiguration setUtilityCachePoolSize(int poolSize) {
         utilityCachePoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -777,9 +807,12 @@ public class IgniteConfiguration {
      * @param keepAliveTime Keep alive time of executor service to use for utility cache messages.
      * @see IgniteConfiguration#getUtilityCacheThreadPoolSize()
      * @see IgniteConfiguration#getUtilityCacheKeepAliveTime()
+     * @return {@code this} for chaining.
      */
-    public void setUtilityCacheKeepAliveTime(long keepAliveTime) {
+    public IgniteConfiguration setUtilityCacheKeepAliveTime(long keepAliveTime) {
         utilityCacheKeepAliveTime = keepAliveTime;
+
+        return this;
     }
 
     /**
@@ -788,9 +821,12 @@ public class IgniteConfiguration {
      * @param poolSize Default executor service size to use for marshaller messages.
      * @see IgniteConfiguration#getMarshallerCacheThreadPoolSize()
      * @see IgniteConfiguration#getMarshallerCacheKeepAliveTime()
+     * @return {@code this} for chaining.
      */
-    public void setMarshallerCachePoolSize(int poolSize) {
+    public IgniteConfiguration setMarshallerCachePoolSize(int poolSize) {
         marshCachePoolSize = poolSize;
+
+        return this;
     }
 
     /**
@@ -799,9 +835,12 @@ public class IgniteConfiguration {
      * @param keepAliveTime Keep alive time of executor service to use for marshaller messages.
      * @see IgniteConfiguration#getMarshallerCacheThreadPoolSize()
      * @see IgniteConfiguration#getMarshallerCacheKeepAliveTime()
+     * @return {@code this} for chaining.
      */
-    public void setMarshallerCacheKeepAliveTime(long keepAliveTime) {
+    public IgniteConfiguration setMarshallerCacheKeepAliveTime(long keepAliveTime) {
         marshCacheKeepAliveTime = keepAliveTime;
+
+        return this;
     }
 
     /**
@@ -824,9 +863,12 @@ public class IgniteConfiguration {
      * @param ggHome {@code Ignition} installation folder.
      * @see IgniteConfiguration#getIgniteHome()
      * @see IgniteSystemProperties#IGNITE_HOME
+     * @return {@code this} for chaining.
      */
-    public void setIgniteHome(String ggHome) {
+    public IgniteConfiguration setIgniteHome(String ggHome) {
         this.ggHome = ggHome;
+
+        return this;
     }
 
     /**
@@ -849,9 +891,12 @@ public class IgniteConfiguration {
      *
      * @param ggWork {@code Ignite} work folder.
      * @see IgniteConfiguration#getWorkDirectory()
+     * @return {@code this} for chaining.
      */
-    public void setWorkDirectory(String ggWork) {
+    public IgniteConfiguration setWorkDirectory(String ggWork) {
         this.ggWork = ggWork;
+
+        return this;
     }
 
     /**
@@ -869,9 +914,12 @@ public class IgniteConfiguration {
      * Sets initialized and started MBean server.
      *
      * @param mbeanSrv Initialized and started MBean server.
+     * @return {@code this} for chaining.
      */
-    public void setMBeanServer(MBeanServer mbeanSrv) {
+    public IgniteConfiguration setMBeanServer(MBeanServer mbeanSrv) {
         this.mbeanSrv = mbeanSrv;
+
+        return this;
     }
 
     /**
@@ -888,9 +936,12 @@ public class IgniteConfiguration {
      *
      * @param nodeId Unique identifier for local node.
      * @see IgniteConfiguration#getNodeId()
+     * @return {@code this} for chaining.
      */
-    public void setNodeId(UUID nodeId) {
+    public IgniteConfiguration setNodeId(UUID nodeId) {
         this.nodeId = nodeId;
+
+        return this;
     }
 
     /**
@@ -909,9 +960,12 @@ public class IgniteConfiguration {
      *
      * @param marsh Marshaller to use within grid.
      * @see IgniteConfiguration#getMarshaller()
+     * @return {@code this} for chaining.
      */
-    public void setMarshaller(Marshaller marsh) {
+    public IgniteConfiguration setMarshaller(Marshaller marsh) {
         this.marsh = marsh;
+
+        return this;
     }
 
 
@@ -949,9 +1003,12 @@ public class IgniteConfiguration {
      * Sets marshal local jobs flag.
      *
      * @param marshLocJobs {@code True} if local jobs should be marshalled.
+     * @return {@code this} for chaining.
      */
-    public void setMarshalLocalJobs(boolean marshLocJobs) {
+    public IgniteConfiguration setMarshalLocalJobs(boolean marshLocJobs) {
         this.marshLocJobs = marshLocJobs;
+
+        return this;
     }
 
     /**
@@ -959,9 +1016,12 @@ public class IgniteConfiguration {
      *
      * @param p2pEnabled {@code true} if peer class loading is
      *      enabled, {@code false} otherwise.
+     * @return {@code this} for chaining.
      */
-    public void setPeerClassLoadingEnabled(boolean p2pEnabled) {
+    public IgniteConfiguration setPeerClassLoadingEnabled(boolean p2pEnabled) {
         this.p2pEnabled = p2pEnabled;
+
+        return this;
     }
 
     /**
@@ -983,9 +1043,12 @@ public class IgniteConfiguration {
      *
      * @param p2pLocClsPathExcl List of P2P loaded packages. Package
      *      name supports '*' at the end like in package import clause.
+     * @return {@code this} for chaining.
      */
-    public void setPeerClassLoadingLocalClassPathExclude(String... p2pLocClsPathExcl) {
+    public IgniteConfiguration setPeerClassLoadingLocalClassPathExclude(String... p2pLocClsPathExcl) {
         this.p2pLocClsPathExcl = p2pLocClsPathExcl;
+
+        return this;
     }
 
     /**
@@ -1007,9 +1070,12 @@ public class IgniteConfiguration {
      * @param metricsHistSize Number of metrics kept in history to use for
      *      metric totals and averages calculations.
      * @see #DFLT_METRICS_HISTORY_SIZE
+     * @return {@code this} for chaining.
      */
-    public void setMetricsHistorySize(int metricsHistSize) {
+    public IgniteConfiguration setMetricsHistorySize(int metricsHistSize) {
         this.metricsHistSize = metricsHistSize;
+
+        return this;
     }
 
     /**
@@ -1043,9 +1109,12 @@ public class IgniteConfiguration {
      * {@link #DFLT_METRICS_UPDATE_FREQ} is used.
      *
      * @param metricsUpdateFreq Job metrics update frequency in milliseconds.
+     * @return {@code this} for chaining.
      */
-    public void setMetricsUpdateFrequency(long metricsUpdateFreq) {
+    public IgniteConfiguration setMetricsUpdateFrequency(long metricsUpdateFreq) {
         this.metricsUpdateFreq = metricsUpdateFreq;
+
+        return this;
     }
 
     /**
@@ -1066,9 +1135,12 @@ public class IgniteConfiguration {
      *
      * @param metricsExpTime The metricsExpTime to set.
      * @see #DFLT_METRICS_EXPIRE_TIME
+     * @return {@code this} for chaining.
      */
-    public void setMetricsExpireTime(long metricsExpTime) {
+    public IgniteConfiguration setMetricsExpireTime(long metricsExpTime) {
         this.metricsExpTime = metricsExpTime;
+
+        return this;
     }
 
     /**
@@ -1092,9 +1164,12 @@ public class IgniteConfiguration {
      *
      * @param netTimeout Maximum timeout for network requests.
      * @see #DFLT_NETWORK_TIMEOUT
+     * @return {@code this} for chaining.
      */
-    public void setNetworkTimeout(long netTimeout) {
+    public IgniteConfiguration setNetworkTimeout(long netTimeout) {
         this.netTimeout = netTimeout;
+
+        return this;
     }
 
     /**
@@ -1118,9 +1193,12 @@ public class IgniteConfiguration {
      * {@link #DFLT_SEND_RETRY_DELAY} is used.
      *
      * @param sndRetryDelay Interval between message send retries.
+     * @return {@code this} for chaining.
      */
-    public void setNetworkSendRetryDelay(long sndRetryDelay) {
+    public IgniteConfiguration setNetworkSendRetryDelay(long sndRetryDelay) {
         this.sndRetryDelay = sndRetryDelay;
+
+        return this;
     }
 
     /**
@@ -1144,9 +1222,12 @@ public class IgniteConfiguration {
      * {@link #DFLT_SEND_RETRY_CNT} is used.
      *
      * @param sndRetryCnt Message send retries count.
+     * @return {@code this} for chaining.
      */
-    public void setNetworkSendRetryCount(int sndRetryCnt) {
+    public IgniteConfiguration setNetworkSendRetryCount(int sndRetryCnt) {
         this.sndRetryCnt = sndRetryCnt;
+
+        return this;
     }
 
     /**
@@ -1164,9 +1245,12 @@ public class IgniteConfiguration {
      * Sets number of samples used for clock synchronization.
      *
      * @param clockSyncSamples Number of samples.
+     * @return {@code this} for chaining.
      */
-    public void setClockSyncSamples(int clockSyncSamples) {
+    public IgniteConfiguration setClockSyncSamples(int clockSyncSamples) {
         this.clockSyncSamples = clockSyncSamples;
+
+        return this;
     }
 
     /**
@@ -1184,9 +1268,12 @@ public class IgniteConfiguration {
      * Sets clock synchronization frequency in milliseconds.
      *
      * @param clockSyncFreq Clock synchronization frequency.
+     * @return {@code this} for chaining.
      */
-    public void setClockSyncFrequency(long clockSyncFreq) {
+    public IgniteConfiguration setClockSyncFrequency(long clockSyncFreq) {
         this.clockSyncFreq = clockSyncFreq;
+
+        return this;
     }
 
     /**
@@ -1211,9 +1298,12 @@ public class IgniteConfiguration {
      *
      * @param lifecycleBeans Collection of lifecycle beans.
      * @see LifecycleEventType
+     * @return {@code this} for chaining.
      */
-    public void setLifecycleBeans(LifecycleBean... lifecycleBeans) {
+    public IgniteConfiguration setLifecycleBeans(LifecycleBean... lifecycleBeans) {
         this.lifecycleBeans = lifecycleBeans;
+
+        return this;
     }
 
     /**
@@ -1231,9 +1321,12 @@ public class IgniteConfiguration {
      *
      * @param evtSpi Fully configured instance of {@link EventStorageSpi}.
      * @see IgniteConfiguration#getEventStorageSpi()
+     * @return {@code this} for chaining.
      */
-    public void setEventStorageSpi(EventStorageSpi evtSpi) {
+    public IgniteConfiguration setEventStorageSpi(EventStorageSpi evtSpi) {
         this.evtSpi = evtSpi;
+
+        return this;
     }
 
     /**
@@ -1251,9 +1344,12 @@ public class IgniteConfiguration {
      *
      * @param discoSpi Fully configured instance of {@link DiscoverySpi}.
      * @see IgniteConfiguration#getDiscoverySpi()
+     * @return {@code this} for chaining.
      */
-    public void setDiscoverySpi(DiscoverySpi discoSpi) {
+    public IgniteConfiguration setDiscoverySpi(DiscoverySpi discoSpi) {
         this.discoSpi = discoSpi;
+
+        return this;
     }
 
     /**
@@ -1269,9 +1365,12 @@ public class IgniteConfiguration {
      * Sets segmentation policy.
      *
      * @param segPlc Segmentation policy.
+     * @return {@code this} for chaining.
      */
-    public void setSegmentationPolicy(SegmentationPolicy segPlc) {
+    public IgniteConfiguration setSegmentationPolicy(SegmentationPolicy segPlc) {
         this.segPlc = segPlc;
+
+        return this;
     }
 
     /**
@@ -1293,9 +1392,12 @@ public class IgniteConfiguration {
      * Sets wait for segment on start flag.
      *
      * @param waitForSegOnStart {@code True} to wait for segment on start.
+     * @return {@code this} for chaining.
      */
-    public void setWaitForSegmentOnStart(boolean waitForSegOnStart) {
+    public IgniteConfiguration setWaitForSegmentOnStart(boolean waitForSegOnStart) {
         this.waitForSegOnStart = waitForSegOnStart;
+
+        return this;
     }
 
     /**
@@ -1320,9 +1422,12 @@ public class IgniteConfiguration {
      *
      * @param allResolversPassReq {@code True} if all segmentation resolvers should
      *      succeed for node to be in the correct segment.
+     * @return {@code this} for chaining.
      */
-    public void setAllSegmentationResolversPassRequired(boolean allResolversPassReq) {
+    public IgniteConfiguration setAllSegmentationResolversPassRequired(boolean allResolversPassReq) {
         this.allResolversPassReq = allResolversPassReq;
+
+        return this;
     }
 
     /**
@@ -1341,9 +1446,12 @@ public class IgniteConfiguration {
      * Sets segmentation resolve attempts count.
      *
      * @param segResolveAttempts Segmentation resolve attempts.
+     * @return {@code this} for chaining.
      */
-    public void setSegmentationResolveAttempts(int segResolveAttempts) {
+    public IgniteConfiguration setSegmentationResolveAttempts(int segResolveAttempts) {
         this.segResolveAttempts = segResolveAttempts;
+
+        return this;
     }
 
     /**
@@ -1362,9 +1470,12 @@ public class IgniteConfiguration {
      * Sets segmentation resolvers.
      *
      * @param segResolvers Segmentation resolvers.
+     * @return {@code this} for chaining.
      */
-    public void setSegmentationResolvers(SegmentationResolver... segResolvers) {
+    public IgniteConfiguration setSegmentationResolvers(SegmentationResolver... segResolvers) {
         this.segResolvers = segResolvers;
+
+        return this;
     }
 
     /**
@@ -1385,9 +1496,12 @@ public class IgniteConfiguration {
      * Sets network segment check frequency.
      *
      * @param segChkFreq Segment check frequency.
+     * @return {@code this} for chaining.
      */
-    public void setSegmentCheckFrequency(long segChkFreq) {
+    public IgniteConfiguration setSegmentCheckFrequency(long segChkFreq) {
         this.segChkFreq = segChkFreq;
+
+        return this;
     }
 
     /**
@@ -1405,9 +1519,12 @@ public class IgniteConfiguration {
      *
      * @param commSpi Fully configured instance of {@link CommunicationSpi}.
      * @see IgniteConfiguration#getCommunicationSpi()
+     * @return {@code this} for chaining.
      */
-    public void setCommunicationSpi(CommunicationSpi commSpi) {
+    public IgniteConfiguration setCommunicationSpi(CommunicationSpi commSpi) {
         this.commSpi = commSpi;
+
+        return this;
     }
 
     /**
@@ -1430,9 +1547,12 @@ public class IgniteConfiguration {
      * @param colSpi Fully configured instance of {@link CollisionSpi} or
      *      {@code null} if no SPI provided.
      * @see IgniteConfiguration#getCollisionSpi()
+     * @return {@code this} for chaining.
      */
-    public void setCollisionSpi(CollisionSpi colSpi) {
+    public IgniteConfiguration setCollisionSpi(CollisionSpi colSpi) {
         this.colSpi = colSpi;
+
+        return this;
     }
 
     /**
@@ -1450,9 +1570,12 @@ public class IgniteConfiguration {
      *
      * @param deploySpi Fully configured instance of {@link DeploymentSpi}.
      * @see IgniteConfiguration#getDeploymentSpi()
+     * @return {@code this} for chaining.
      */
-    public void setDeploymentSpi(DeploymentSpi deploySpi) {
+    public IgniteConfiguration setDeploymentSpi(DeploymentSpi deploySpi) {
         this.deploySpi = deploySpi;
+
+        return this;
     }
 
     /**
@@ -1470,9 +1593,12 @@ public class IgniteConfiguration {
      *
      * @param cpSpi Fully configured instance of {@link CheckpointSpi}.
      * @see IgniteConfiguration#getCheckpointSpi()
+     * @return {@code this} for chaining.
      */
-    public void setCheckpointSpi(CheckpointSpi... cpSpi) {
+    public IgniteConfiguration setCheckpointSpi(CheckpointSpi... cpSpi) {
         this.cpSpi = cpSpi;
+
+        return this;
     }
 
     /**
@@ -1491,9 +1617,12 @@ public class IgniteConfiguration {
      * @param failSpi Fully configured instance of {@link FailoverSpi} or
      *      {@code null} if no SPI provided.
      * @see IgniteConfiguration#getFailoverSpi()
+     * @return {@code this} for chaining.
      */
-    public void setFailoverSpi(FailoverSpi... failSpi) {
+    public IgniteConfiguration setFailoverSpi(FailoverSpi... failSpi) {
         this.failSpi = failSpi;
+
+        return this;
     }
 
     /**
@@ -1533,9 +1662,12 @@ public class IgniteConfiguration {
      *
      * @param discoStartupDelay Time in milliseconds for when nodes
      *      can be out-of-sync during startup.
+     * @return {@code this} for chaining.
      */
-    public void setDiscoveryStartupDelay(long discoStartupDelay) {
+    public IgniteConfiguration setDiscoveryStartupDelay(long discoStartupDelay) {
         this.discoStartupDelay = discoStartupDelay;
+
+        return this;
     }
 
     /**
@@ -1544,9 +1676,12 @@ public class IgniteConfiguration {
      * @param loadBalancingSpi Fully configured instance of {@link LoadBalancingSpi} or
      *      {@code null} if no SPI provided.
      * @see IgniteConfiguration#getLoadBalancingSpi()
+     * @return {@code this} for chaining.
      */
-    public void setLoadBalancingSpi(LoadBalancingSpi... loadBalancingSpi) {
+    public IgniteConfiguration setLoadBalancingSpi(LoadBalancingSpi... loadBalancingSpi) {
         this.loadBalancingSpi = loadBalancingSpi;
+
+        return this;
     }
 
     /**
@@ -1555,9 +1690,12 @@ public class IgniteConfiguration {
      * @param swapSpaceSpi Fully configured instances of {@link SwapSpaceSpi} or
      *      <tt>null</tt> if no SPI provided.
      * @see IgniteConfiguration#getSwapSpaceSpi()
+     * @return {@code this} for chaining.
      */
-    public void setSwapSpaceSpi(SwapSpaceSpi swapSpaceSpi) {
+    public IgniteConfiguration setSwapSpaceSpi(SwapSpaceSpi swapSpaceSpi) {
         this.swapSpaceSpi = swapSpaceSpi;
+
+        return this;
     }
 
     /**
@@ -1578,9 +1716,12 @@ public class IgniteConfiguration {
      *
      * @param indexingSpi Fully configured instance of {@link IndexingSpi}.
      * @see IgniteConfiguration#getIndexingSpi()
+     * @return {@code this} for chaining.
      */
-    public void setIndexingSpi(IndexingSpi indexingSpi) {
+    public IgniteConfiguration setIndexingSpi(IndexingSpi indexingSpi) {
         this.indexingSpi = indexingSpi;
+
+        return this;
     }
 
     /**
@@ -1605,18 +1746,24 @@ public class IgniteConfiguration {
      * Sets address resolver for addresses mapping determination.
      *
      * @param addrRslvr Address resolver.
+     * @return {@code this} for chaining.
      */
-    public void setAddressResolver(AddressResolver addrRslvr) {
+    public IgniteConfiguration setAddressResolver(AddressResolver addrRslvr) {
         this.addrRslvr = addrRslvr;
+
+        return this;
     }
 
     /**
      * Sets task classes and resources sharing mode.
      *
      * @param deployMode Task classes and resources sharing mode.
+     * @return {@code this} for chaining.
      */
-    public void setDeploymentMode(DeploymentMode deployMode) {
+    public IgniteConfiguration setDeploymentMode(DeploymentMode deployMode) {
         this.deployMode = deployMode;
+
+        return this;
     }
 
     /**
@@ -1634,9 +1781,12 @@ public class IgniteConfiguration {
      * missed resources caching.
      *
      * @param p2pMissedCacheSize Size of missed resources cache.
+     * @return {@code this} for chaining.
      */
-    public void setPeerClassLoadingMissedResourcesCacheSize(int p2pMissedCacheSize) {
+    public IgniteConfiguration setPeerClassLoadingMissedResourcesCacheSize(int p2pMissedCacheSize) {
         this.p2pMissedCacheSize = p2pMissedCacheSize;
+
+        return this;
     }
 
     /**
@@ -1666,8 +1816,10 @@ public class IgniteConfiguration {
      * @param cacheCfg Cache configurations.
      */
     @SuppressWarnings({"ZeroLengthArrayAllocation"})
-    public void setCacheConfiguration(CacheConfiguration... cacheCfg) {
+    public IgniteConfiguration setCacheConfiguration(CacheConfiguration... cacheCfg) {
         this.cacheCfg = cacheCfg == null ? new CacheConfiguration[0] : cacheCfg;
+
+        return this;
     }
 
     /**
@@ -1683,9 +1835,12 @@ public class IgniteConfiguration {
      * Sets client mode flag.
      *
      * @param clientMode Client mode flag.
+     * @return {@code this} for chaining.
      */
-    public void setClientMode(boolean clientMode) {
+    public IgniteConfiguration setClientMode(boolean clientMode) {
         this.clientMode = clientMode;
+
+        return this;
     }
 
     /**
@@ -1716,9 +1871,12 @@ public class IgniteConfiguration {
      *
      * @param cacheSanityCheckEnabled {@code True} if cache sanity check is enabled.
      * @see #isCacheSanityCheckEnabled()
+     * @return {@code this} for chaining.
      */
-    public void setCacheSanityCheckEnabled(boolean cacheSanityCheckEnabled) {
+    public IgniteConfiguration setCacheSanityCheckEnabled(boolean cacheSanityCheckEnabled) {
         this.cacheSanityCheckEnabled = cacheSanityCheckEnabled;
+
+        return this;
     }
 
     /**
@@ -1743,9 +1901,12 @@ public class IgniteConfiguration {
      * Note, that either the include event types or the exclude event types can be established.
      *
      * @param inclEvtTypes Include event types.
+     * @return {@code this} for chaining.
      */
-    public void setIncludeEventTypes(int... inclEvtTypes) {
+    public IgniteConfiguration setIncludeEventTypes(int... inclEvtTypes) {
         this.inclEvtTypes = inclEvtTypes;
+
+        return this;
     }
 
     /**
@@ -1753,9 +1914,12 @@ public class IgniteConfiguration {
      * override all default local bind settings within Ignite or any of its SPIs.
      *
      * @param locHost Local IP address or host to bind to.
+     * @return {@code this} for chaining.
      */
-    public void setLocalHost(String locHost) {
+    public IgniteConfiguration setLocalHost(String locHost) {
         this.locHost = locHost;
+
+        return this;
     }
 
     /**
@@ -1791,9 +1955,12 @@ public class IgniteConfiguration {
      * Sets time server port base.
      *
      * @param timeSrvPortBase Time server port base.
+     * @return {@code this} for chaining.
      */
-    public void setTimeServerPortBase(int timeSrvPortBase) {
+    public IgniteConfiguration setTimeServerPortBase(int timeSrvPortBase) {
         this.timeSrvPortBase = timeSrvPortBase;
+
+        return this;
     }
 
     /**
@@ -1809,9 +1976,12 @@ public class IgniteConfiguration {
      * Sets time server port range.
      *
      * @param timeSrvPortRange Time server port range.
+     * @return {@code this} for chaining.
      */
-    public void setTimeServerPortRange(int timeSrvPortRange) {
+    public IgniteConfiguration setTimeServerPortRange(int timeSrvPortRange) {
         this.timeSrvPortRange = timeSrvPortRange;
+
+        return this;
     }
 
     /**
@@ -1832,9 +2002,12 @@ public class IgniteConfiguration {
      * See {@link #getIncludeProperties()} for more info.
      *
      * @param includeProps Array of system or environment property names to include into node attributes.
+     * @return {@code this} for chaining.
      */
-    public void setIncludeProperties(String... includeProps) {
+    public IgniteConfiguration setIncludeProperties(String... includeProps) {
         this.includeProps = includeProps;
+
+        return this;
     }
 
     /**
@@ -1858,9 +2031,12 @@ public class IgniteConfiguration {
      * If not provided, then default value {@link #DFLT_METRICS_LOG_FREQ} is used.
      *
      * @param metricsLogFreq Frequency of metrics log print out.
+     * @return {@code this} for chaining.
      */
-    public void setMetricsLogFrequency(long metricsLogFreq) {
+    public IgniteConfiguration setMetricsLogFrequency(long metricsLogFreq) {
         this.metricsLogFreq = metricsLogFreq;
+
+        return this;
     }
 
     /**
@@ -1876,9 +2052,12 @@ public class IgniteConfiguration {
      * Sets IGFS (Ignite In-Memory File System) configurations.
      *
      * @param igfsCfg IGFS configurations.
+     * @return {@code this} for chaining.
      */
-    public void setFileSystemConfiguration(FileSystemConfiguration... igfsCfg) {
+    public IgniteConfiguration setFileSystemConfiguration(FileSystemConfiguration... igfsCfg) {
         this.igfsCfg = igfsCfg;
+
+        return this;
     }
 
     /**
@@ -1894,9 +2073,12 @@ public class IgniteConfiguration {
      * Sets hadoop configuration.
      *
      * @param hadoopCfg Hadoop configuration.
+     * @return {@code this} for chaining.
      */
-    public void setHadoopConfiguration(HadoopConfiguration hadoopCfg) {
+    public IgniteConfiguration setHadoopConfiguration(HadoopConfiguration hadoopCfg) {
         this.hadoopCfg = hadoopCfg;
+
+        return this;
     }
 
     /**
@@ -1908,9 +2090,12 @@ public class IgniteConfiguration {
 
     /**
      * @param connectorCfg Connector configuration.
+     * @return {@code this} for chaining.
      */
-    public void setConnectorConfiguration(ConnectorConfiguration connectorCfg) {
+    public IgniteConfiguration setConnectorConfiguration(ConnectorConfiguration connectorCfg) {
         this.connectorCfg = connectorCfg;
+
+        return this;
     }
 
     /**
@@ -1926,9 +2111,12 @@ public class IgniteConfiguration {
      * Sets configurations for services to be deployed on the grid.
      *
      * @param svcCfgs Configurations for services to be deployed on the grid.
+     * @return {@code this} for chaining.
      */
-    public void setServiceConfiguration(ServiceConfiguration... svcCfgs) {
+    public IgniteConfiguration setServiceConfiguration(ServiceConfiguration... svcCfgs) {
         this.svcCfgs = svcCfgs;
+
+        return this;
     }
 
     /**
@@ -1947,9 +2135,12 @@ public class IgniteConfiguration {
      * Each listener is mapped to array of event types.
      *
      * @param lsnrs Pre-configured event listeners map.
+     * @return {@code this} for chaining.
      */
-    public void setLocalEventListeners(Map<IgnitePredicate<? extends Event>, int[]> lsnrs) {
+    public IgniteConfiguration setLocalEventListeners(Map<IgnitePredicate<? extends Event>, int[]> lsnrs) {
         this.lsnrs = lsnrs;
+
+        return this;
     }
 
     /**
@@ -1967,9 +2158,12 @@ public class IgniteConfiguration {
      *
      * @param warmupClos Warmup closure to execute.
      * @see #getWarmupClosure()
+     * @return {@code this} for chaining.
      */
-    public void setWarmupClosure(IgniteInClosure<IgniteConfiguration> warmupClos) {
+    public IgniteConfiguration setWarmupClosure(IgniteInClosure<IgniteConfiguration> warmupClos) {
         this.warmupClos = warmupClos;
+
+        return this;
     }
 
     /**
@@ -1985,9 +2179,12 @@ public class IgniteConfiguration {
      * Sets transactions configuration.
      *
      * @param txCfg Transactions configuration.
+     * @return {@code this} for chaining.
      */
-    public void setTransactionConfiguration(TransactionConfiguration txCfg) {
+    public IgniteConfiguration setTransactionConfiguration(TransactionConfiguration txCfg) {
         this.txCfg = txCfg;
+
+        return this;
     }
 
     /**
@@ -1999,9 +2196,12 @@ public class IgniteConfiguration {
 
     /**
      * @param pluginCfgs Plugin configurations.
+     * @return {@code this} for chaining.
      */
-    public void setPluginConfigurations(PluginConfiguration... pluginCfgs) {
+    public IgniteConfiguration setPluginConfigurations(PluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
+
+        return this;
     }
 
     /**
@@ -2013,9 +2213,12 @@ public class IgniteConfiguration {
 
     /**
      * @param atomicCfg Atomic data structures configuration.
+     * @return {@code this} for chaining.
      */
-    public void setAtomicConfiguration(AtomicConfiguration atomicCfg) {
+    public IgniteConfiguration setAtomicConfiguration(AtomicConfiguration atomicCfg) {
         this.atomicCfg = atomicCfg;
+
+        return this;
     }
 
     /**
@@ -2024,9 +2227,12 @@ public class IgniteConfiguration {
      * {@link ExpiryPolicy ExpiryPolicys}).
      *
      * @param classLdr Class loader.
+     * @return {@code this} for chaining.
      */
-    public void setClassLoader(ClassLoader classLdr) {
+    public IgniteConfiguration setClassLoader(ClassLoader classLdr) {
         this.classLdr = classLdr;
+
+        return this;
     }
 
     /**


[02/50] incubator-ignite git commit: # ignite-791 schema-import profile.

Posted by sb...@apache.org.
# ignite-791 schema-import profile.


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

Branch: refs/heads/ignite-834
Commit: 03012075cb7ae855bc23e44c7ed9e04ba37e7208
Parents: 0f3d372
Author: Andrey <an...@gridgain.com>
Authored: Wed Apr 29 13:11:10 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Apr 29 13:11:10 2015 +0700

----------------------------------------------------------------------
 pom.xml | 63 ++++++++++++++++++++++++++++++++++++++++--------------------
 1 file changed, 42 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03012075/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index b8acf97..5c232de 100644
--- a/pom.xml
+++ b/pom.xml
@@ -74,7 +74,6 @@
         <module>modules/log4j</module>
         <module>modules/slf4j</module>
         <module>modules/jcl</module>
-        <module>modules/schema-import</module>
         <module>modules/codegen</module>
     </modules>
 
@@ -132,10 +131,6 @@
                                         <fileset>
                                             <directory>libs</directory>
                                         </fileset>
-
-                                        <fileset>
-                                            <directory>bin/include/schema-import</directory>
-                                        </fileset>
                                     </filesets>
                                 </configuration>
                             </execution>
@@ -381,22 +376,6 @@
                             </execution>
 
                             <execution>
-                                <id>dependencies-schema-import</id>
-                                <phase>prepare-package</phase>
-                                <goals>
-                                    <goal>single</goal>
-                                </goals>
-                                <configuration>
-                                    <descriptors>
-                                        <descriptor>assembly/dependencies-schema-import.xml</descriptor>
-                                    </descriptors>
-                                    <outputDirectory>target/release-package/bin</outputDirectory>
-                                    <finalName>include</finalName>
-                                    <appendAssemblyId>false</appendAssemblyId>
-                                </configuration>
-                            </execution>
-
-                            <execution>
                                 <id>dependencies-visor-console</id>
                                 <phase>prepare-package</phase>
                                 <goals>
@@ -569,5 +548,47 @@
                 </plugins>
             </build>
         </profile>
+
+        <profile>
+            <id>schema-import</id>
+            <activation>
+                <jdk>[1.8)</jdk>
+                <property>
+                    <name>java.vendor</name>
+                    <value>Oracle Corporation</value>
+                </property>
+            </activation>
+            <modules>
+                <module>modules/schema-import</module>
+            </modules>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-assembly-plugin</artifactId>
+                        <version>2.4</version>
+                        <inherited>false</inherited>
+
+                        <executions>
+                            <execution>
+                                <id>dependencies-schema-import</id>
+                                <phase>prepare-package</phase>
+                                <goals>
+                                    <goal>single</goal>
+                                </goals>
+                                <configuration>
+                                    <descriptors>
+                                        <descriptor>assembly/dependencies-schema-import.xml</descriptor>
+                                    </descriptors>
+                                    <outputDirectory>target/release-package/bin</outputDirectory>
+                                    <finalName>include</finalName>
+                                    <appendAssemblyId>false</appendAssemblyId>
+                                </configuration>
+                            </execution>
+                        </executions>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
     </profiles>
 </project>


[28/50] incubator-ignite git commit: Test renamed

Posted by sb...@apache.org.
Test renamed


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

Branch: refs/heads/ignite-834
Commit: ee5bae9ed9b7fe328ba216b52a6b2e297881ddb8
Parents: 38dee89
Author: agura <ag...@gridgain.com>
Authored: Thu Apr 30 12:04:56 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Apr 30 12:04:56 2015 +0300

----------------------------------------------------------------------
 ...CacheLoadingConcurrentGridStartSelfTest.java | 154 +++++++++++++++++++
 ...GridCacheLoadingConcurrentGridStartTest.java | 154 -------------------
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 3 files changed, 155 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee5bae9e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
new file mode 100644
index 0000000..74273d1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLoadingConcurrentGridStartSelfTest.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests for cache data loading during simultaneous grids start.
+ */
+public class CacheLoadingConcurrentGridStartSelfTest extends GridCommonAbstractTest {
+    /** Grids count */
+    private static int GRIDS_CNT = 5;
+
+    /** Keys count */
+    private static int KEYS_CNT = 1_000_000;
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+
+        ccfg.setBackups(1);
+
+        CacheStore<Integer, String> store = new CacheStoreAdapter<Integer, String>() {
+            @Override public void loadCache(IgniteBiInClosure<Integer, String> f, Object... args) {
+                for (int i = 0; i < KEYS_CNT; i++)
+                    f.apply(i, Integer.toString(i));
+            }
+
+            @Nullable @Override public String load(Integer i) throws CacheLoaderException {
+                return null;
+            }
+
+            @Override public void write(Cache.Entry<? extends Integer, ? extends String> entry) throws CacheWriterException {
+                // No-op.
+            }
+
+            @Override public void delete(Object o) throws CacheWriterException {
+                // No-op.
+            }
+        };
+
+        ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(store));
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheWithDataStreamer() throws Exception {
+        IgniteInClosure<Ignite> f = new IgniteInClosure<Ignite>() {
+            @Override public void apply(Ignite grid) {
+                try (IgniteDataStreamer<Integer, String> dataStreamer = grid.dataStreamer(null)) {
+                    for (int i = 0; i < KEYS_CNT; i++)
+                        dataStreamer.addData(i, Integer.toString(i));
+                }
+            }
+        };
+
+        loadCache(f);
+    }
+
+    /**
+     * @throws Exception if failed
+     */
+    public void testLoadCacheFromStore() throws Exception {
+        loadCache(new IgniteInClosure<Ignite>() {
+            @Override public void apply(Ignite grid) {
+                grid.cache(null).loadCache(null);
+            }
+        });
+    }
+
+    /**
+     * Loads cache using closure and asserts cache size.
+     *
+     * @param f cache loading closure
+     * @throws Exception if failed
+     */
+    private void loadCache(IgniteInClosure<Ignite> f) throws Exception {
+        Ignite g0 = startGrid(0);
+
+        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Ignite>() {
+            @Override public Ignite call() throws Exception {
+                return startGridsMultiThreaded(1, GRIDS_CNT - 1);
+            }
+        });
+
+        try {
+            f.apply(g0);
+        }
+        finally {
+            fut.get();
+        }
+
+        assertCacheSize();
+    }
+
+    /** Asserts cache size. */
+    private void assertCacheSize() {
+        IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+        assertEquals(KEYS_CNT, cache.size(CachePeekMode.PRIMARY));
+
+        int total = 0;
+
+        for (int i = 0; i < GRIDS_CNT; i++)
+            total += grid(i).cache(null).localSize(CachePeekMode.PRIMARY);
+
+        assertEquals(KEYS_CNT, total);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee5bae9e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLoadingConcurrentGridStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLoadingConcurrentGridStartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLoadingConcurrentGridStartTest.java
deleted file mode 100644
index 2f9bb96..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLoadingConcurrentGridStartTest.java
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.store.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.*;
-import javax.cache.configuration.*;
-import javax.cache.integration.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests for cache data loading during simultaneous grids start.
- */
-public class GridCacheLoadingConcurrentGridStartTest extends GridCommonAbstractTest {
-    /** Grids count */
-    private static int GRIDS_CNT = 5;
-
-    /** Keys count */
-    private static int KEYS_CNT = 1_000_000;
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration ccfg = new CacheConfiguration();
-
-        ccfg.setCacheMode(PARTITIONED);
-
-        ccfg.setBackups(1);
-
-        CacheStore<Integer, String> store = new CacheStoreAdapter<Integer, String>() {
-            @Override public void loadCache(IgniteBiInClosure<Integer, String> f, Object... args) {
-                for (int i = 0; i < KEYS_CNT; i++)
-                    f.apply(i, Integer.toString(i));
-            }
-
-            @Nullable @Override public String load(Integer i) throws CacheLoaderException {
-                return null;
-            }
-
-            @Override public void write(Cache.Entry<? extends Integer, ? extends String> entry) throws CacheWriterException {
-                // No-op.
-            }
-
-            @Override public void delete(Object o) throws CacheWriterException {
-                // No-op.
-            }
-        };
-
-        ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(store));
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * @throws Exception if failed
-     */
-    public void testLoadCacheWithDataStreamer() throws Exception {
-        IgniteInClosure<Ignite> f = new IgniteInClosure<Ignite>() {
-            @Override public void apply(Ignite grid) {
-                try (IgniteDataStreamer<Integer, String> dataStreamer = grid.dataStreamer(null)) {
-                    for (int i = 0; i < KEYS_CNT; i++)
-                        dataStreamer.addData(i, Integer.toString(i));
-                }
-            }
-        };
-
-        loadCache(f);
-    }
-
-    /**
-     * @throws Exception if failed
-     */
-    public void testLoadCacheFromStore() throws Exception {
-        loadCache(new IgniteInClosure<Ignite>() {
-            @Override public void apply(Ignite grid) {
-                grid.cache(null).loadCache(null);
-            }
-        });
-    }
-
-    /**
-     * Loads cache using closure and asserts cache size.
-     *
-     * @param f cache loading closure
-     * @throws Exception if failed
-     */
-    private void loadCache(IgniteInClosure<Ignite> f) throws Exception {
-        Ignite g0 = startGrid(0);
-
-        IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable<Ignite>() {
-            @Override public Ignite call() throws Exception {
-                return startGridsMultiThreaded(1, GRIDS_CNT - 1);
-            }
-        });
-
-        try {
-            f.apply(g0);
-        }
-        finally {
-            fut.get();
-        }
-
-        assertCacheSize();
-    }
-
-    /** Asserts cache size. */
-    private void assertCacheSize() {
-        IgniteCache<Integer, String> cache = grid(0).cache(null);
-
-        assertEquals(KEYS_CNT, cache.size(CachePeekMode.PRIMARY));
-
-        int total = 0;
-
-        for (int i = 0; i < GRIDS_CNT; i++)
-            total += grid(i).cache(null).localSize(CachePeekMode.PRIMARY);
-
-        assertEquals(KEYS_CNT, total);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ee5bae9e/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 6f954cd..bb019ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -224,7 +224,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheDhtPreloadUnloadSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedAffinityFilterSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedPreloadLifecycleSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheLoadingConcurrentGridStartTest.class));
+        suite.addTest(new TestSuite(CacheLoadingConcurrentGridStartSelfTest.class));
         suite.addTest(new TestSuite(GridCacheDhtPreloadDelayedSelfTest.class));
         suite.addTest(new TestSuite(GridPartitionedBackupLoadSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedLoadCacheSelfTest.class));


[09/50] incubator-ignite git commit: # IGNITE-789 review.

Posted by sb...@apache.org.
# IGNITE-789 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/a06f5faf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a06f5faf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a06f5faf

Branch: refs/heads/ignite-834
Commit: a06f5faf6e0fe509a9fed339c32002834aa672a9
Parents: be7bfda
Author: AKuznetsov <ak...@gridgain.com>
Authored: Wed Apr 29 18:54:55 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Wed Apr 29 18:54:55 2015 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheStartTask.java        | 86 ++++++++++++++++----
 1 file changed, 68 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a06f5faf/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
index 270c48a..2aa03a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStartTask.java
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
 import org.apache.ignite.internal.visor.util.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -36,12 +35,12 @@ import java.util.*;
  */
 @GridInternal
 public class VisorCacheStartTask extends
-    VisorMultiNodeTask<IgniteBiTuple<String, String>, Map<UUID, IgniteException>, Void> {
+    VisorMultiNodeTask<VisorCacheStartTask.VisorCacheStartArg, Map<UUID, IgniteException>, Void> {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** {@inheritDoc} */
-    @Override protected VisorCacheStartJob job(IgniteBiTuple<String, String> arg) {
+    @Override protected VisorCacheStartJob job(VisorCacheStartArg arg) {
         return new VisorCacheStartJob(arg, debug);
     }
 
@@ -57,9 +56,59 @@ public class VisorCacheStartTask extends
     }
 
     /**
+     * Cache start arguments.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class VisorCacheStartArg implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final boolean near;
+
+        /** */
+        private final String name;
+
+        /** */
+        private final String cfg;
+
+        /**
+         * @param near {@code true} if near cache should be started.
+         * @param name Name for near cache.
+         * @param cfg Cache XML configuration.
+         */
+        public VisorCacheStartArg(boolean near, String name, String cfg) {
+            this.near = near;
+            this.name = name;
+            this.cfg = cfg;
+        }
+
+        /**
+         * @return {@code true} if near cache should be started.
+         */
+        public boolean near() {
+            return near;
+        }
+
+        /**
+         * @return Name for near cache.
+         */
+        public String name() {
+            return name;
+        }
+
+        /**
+         * @return Cache XML configuration.
+         */
+        public String configuration() {
+            return cfg;
+        }
+    }
+
+    /**
      * Job that start cache or near cache with specified configuration.
      */
-    private static class VisorCacheStartJob extends VisorJob<IgniteBiTuple<String, String>, Void> {
+    private static class VisorCacheStartJob extends VisorJob<VisorCacheStartArg, Void> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -69,29 +118,30 @@ public class VisorCacheStartTask extends
          * @param arg Contains cache name and XML configurations of cache.
          * @param debug Debug flag.
          */
-        private VisorCacheStartJob(IgniteBiTuple<String, String> arg, boolean debug) {
+        private VisorCacheStartJob(VisorCacheStartArg arg, boolean debug) {
             super(arg, debug);
         }
 
         /** {@inheritDoc} */
-        @Override protected Void run(IgniteBiTuple<String, String> arg) {
-            String name = arg.get1();
-
-            String cfg = arg.get2();
+        @Override protected Void run(VisorCacheStartArg arg) throws IgniteException {
+            String cfg = arg.configuration();
 
             assert !F.isEmpty(cfg);
 
-            if (name == null) {
-                CacheConfiguration cacheCfg = Ignition.loadSpringBean(new ByteArrayInputStream(cfg.getBytes()),
-                    "cacheConfiguration");
+            try (ByteArrayInputStream bais = new ByteArrayInputStream(cfg.getBytes())) {
+                if (arg.near) {
+                    NearCacheConfiguration nearCfg = Ignition.loadSpringBean(bais, "nearCacheConfiguration");
 
-                ignite.createCache(cacheCfg);
-            }
-            else {
-                NearCacheConfiguration nearCfg = Ignition.loadSpringBean(
-                    new ByteArrayInputStream(arg.get2().getBytes()), "nearCacheConfiguration");
+                    ignite.createNearCache(VisorTaskUtils.unescapeName(arg.name()), nearCfg);
+                }
+                else {
+                    CacheConfiguration cacheCfg = Ignition.loadSpringBean(bais, "cacheConfiguration");
 
-                ignite.createNearCache(VisorTaskUtils.unescapeName(name), nearCfg);
+                    ignite.createCache(cacheCfg);
+                }
+            }
+            catch (IOException e) {
+                throw new  IgniteException(e);
             }
 
             return null;


[35/50] incubator-ignite git commit: minor

Posted by sb...@apache.org.
minor


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

Branch: refs/heads/ignite-834
Commit: e0648d68781e3906b66f3c876fef59bc4141454e
Parents: f6ac27a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Apr 30 17:18:44 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Apr 30 17:18:44 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e0648d68/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
index 09cbd97..0c9f2f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
@@ -275,9 +275,12 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * TODO: IGNITE-587.
      * @throws Exception If failed.
      */
     public void testClientReconnect() throws Exception {
+        fail("ignite-587");
+
         clientsPerSrv = 1;
 
         startServerNodes(3);
@@ -318,9 +321,12 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * TODO: IGNITE-587.
      * @throws Exception If failed.
      */
     public void testClientNodeLeaveOneServer() throws Exception {
+        fail("ignite-587");
+
         startServerNodes(1);
         startClientNodes(1);
 
@@ -430,6 +436,8 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDataExchangeFromClient() throws Exception {
+        fail("ignite-587");
+
         testDataExchange("client-0");
     }
 


[26/50] incubator-ignite git commit: Merge branches 'ignite-776' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-776

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


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

Branch: refs/heads/ignite-834
Commit: 81ce0e63926b450ad526ad9dbe8045ec89ae5308
Parents: 8b18e87 1e53395
Author: Andrey <an...@gridgain.com>
Authored: Thu Apr 30 10:07:51 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Apr 30 10:07:51 2015 +0700

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   2 +-
 modules/cloud/README.txt                        |  32 ++
 modules/cloud/licenses/apache-2.0.txt           | 202 +++++++++
 modules/cloud/pom.xml                           | 106 +++++
 .../cloud/TcpDiscoveryCloudIpFinder.java        | 433 +++++++++++++++++++
 .../tcp/ipfinder/cloud/package-info.java        |  21 +
 .../TcpDiscoveryCloudIpFinderSelfTest.java      | 124 ++++++
 .../tcp/ipfinder/cloud/package-info.java        |  22 +
 .../ignite/testsuites/IgniteCloudTestSuite.java | 112 +++++
 .../ignite/codegen/MessageCodeGenerator.java    |  26 +-
 .../configuration/CacheConfiguration.java       | 259 ++++++++---
 .../configuration/IgniteConfiguration.java      | 344 ++++++++++++---
 .../ignite/internal/GridDirectCollection.java   |   3 +
 .../ignite/internal/IgniteComponentType.java    |  36 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  15 +-
 .../managers/communication/GridIoManager.java   |  22 +-
 .../communication/GridIoMessageFactory.java     |   8 +
 .../managers/indexing/GridIndexingManager.java  |  14 +-
 .../processors/cache/CacheObjectImpl.java       |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |   9 +-
 .../processors/cache/GridCacheProcessor.java    |   3 +-
 .../processors/cache/GridCacheSwapManager.java  |  35 +-
 .../cache/query/GridCacheQueryManager.java      |  22 +-
 .../cache/query/GridCacheSqlQuery.java          | 135 +++++-
 .../cache/query/GridCacheTwoStepQuery.java      |   8 +-
 .../processors/query/GridQueryIndexing.java     |  19 +-
 .../processors/query/GridQueryProcessor.java    |  78 +++-
 .../messages/GridQueryNextPageResponse.java     |  68 ++-
 .../h2/twostep/messages/GridQueryRequest.java   |  21 +-
 .../util/spring/IgniteSpringHelper.java         |   4 +-
 .../internal/visor/cache/VisorCacheMetrics.java |  53 +--
 .../cache/VisorCacheNearConfiguration.java      |   4 +-
 .../visor/cache/VisorCacheStartTask.java        | 155 +++++++
 .../internal/visor/util/VisorTaskUtils.java     |  10 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  19 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |  10 +-
 .../resources/META-INF/classnames.properties    |  13 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +-
 modules/gce/README.txt                          |  32 ++
 modules/gce/licenses/apache-2.0.txt             | 202 +++++++++
 modules/gce/pom.xml                             |  92 ++++
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  | 380 ++++++++++++++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 ...pDiscoveryGoogleStorageIpFinderSelfTest.java |  73 ++++
 .../tcp/ipfinder/gce/package-info.java          |  22 +
 .../ignite/testsuites/IgniteGCETestSuite.java   |  71 +++
 .../processors/query/h2/IgniteH2Indexing.java   | 169 +++++++-
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  92 +---
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   7 +-
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   6 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |  14 +-
 .../processors/query/h2/opt/GridH2Table.java    |  10 +-
 .../query/h2/opt/GridH2ValueCacheObject.java    | 191 ++++++++
 .../query/h2/opt/GridLuceneIndex.java           |  84 ++--
 .../query/h2/twostep/GridMapQueryExecutor.java  |  21 +-
 .../query/h2/twostep/GridMergeIndex.java        |   6 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  13 +-
 .../query/h2/twostep/GridResultPage.java        |  80 +++-
 .../query/h2/twostep/msg/GridH2Array.java       | 124 ++++++
 .../query/h2/twostep/msg/GridH2Boolean.java     | 112 +++++
 .../query/h2/twostep/msg/GridH2Byte.java        | 113 +++++
 .../query/h2/twostep/msg/GridH2Bytes.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2CacheObject.java | 148 +++++++
 .../query/h2/twostep/msg/GridH2Date.java        | 115 +++++
 .../query/h2/twostep/msg/GridH2Decimal.java     | 134 ++++++
 .../query/h2/twostep/msg/GridH2Double.java      | 113 +++++
 .../query/h2/twostep/msg/GridH2Float.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2Geometry.java    | 134 ++++++
 .../query/h2/twostep/msg/GridH2Integer.java     | 113 +++++
 .../query/h2/twostep/msg/GridH2JavaObject.java  | 113 +++++
 .../query/h2/twostep/msg/GridH2Long.java        | 113 +++++
 .../query/h2/twostep/msg/GridH2Null.java        |  78 ++++
 .../query/h2/twostep/msg/GridH2Short.java       | 113 +++++
 .../query/h2/twostep/msg/GridH2String.java      | 115 +++++
 .../query/h2/twostep/msg/GridH2Time.java        | 116 +++++
 .../query/h2/twostep/msg/GridH2Timestamp.java   | 133 ++++++
 .../query/h2/twostep/msg/GridH2Uuid.java        | 133 ++++++
 .../h2/twostep/msg/GridH2ValueMessage.java      |  49 +++
 .../twostep/msg/GridH2ValueMessageFactory.java  | 201 +++++++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   6 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     | 130 ++++--
 .../util/spring/IgniteSpringHelperImpl.java     |   2 +-
 ...gniteProjectionStartStopRestartSelfTest.java |  26 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 +-
 pom.xml                                         |   5 +-
 87 files changed, 6182 insertions(+), 529 deletions(-)
----------------------------------------------------------------------



[37/50] incubator-ignite git commit: deploy to site disabled

Posted by sb...@apache.org.
deploy to site disabled


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

Branch: refs/heads/ignite-834
Commit: ea91d220928a0ebb215500141d298059d5547320
Parents: 7121619
Author: avinogradov <av...@gridgain.com>
Authored: Thu Apr 30 18:10:29 2015 +0300
Committer: avinogradov <av...@gridgain.com>
Committed: Thu Apr 30 18:10:29 2015 +0300

----------------------------------------------------------------------
 pom.xml | 46 +++++++++++++++++++++++-----------------------
 1 file changed, 23 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ea91d220/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 98541ef..181540b 100644
--- a/pom.xml
+++ b/pom.xml
@@ -577,29 +577,29 @@
                         </executions>
                     </plugin>
 
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-site-plugin</artifactId>
-                        <version>3.4</version>
-                        <dependencies>
-                            <dependency>
-                                <groupId>org.apache.maven.wagon</groupId>
-                                <artifactId>wagon-ssh</artifactId>
-                                <version>2.8</version>
-                            </dependency>
-                        </dependencies>
-                        <executions>
-                            <execution>
-                                <goals>
-                                    <goal>deploy</goal>
-                                </goals>
-                                <phase>deploy</phase>
-                                <configuration>
-                                    <inputDirectory>${basedir}/target/site</inputDirectory>
-                                </configuration>
-                            </execution>
-                        </executions>
-                    </plugin>
+                    <!--<plugin>-->
+                        <!--<groupId>org.apache.maven.plugins</groupId>-->
+                        <!--<artifactId>maven-site-plugin</artifactId>-->
+                        <!--<version>3.4</version>-->
+                        <!--<dependencies>-->
+                            <!--<dependency>-->
+                                <!--<groupId>org.apache.maven.wagon</groupId>-->
+                                <!--<artifactId>wagon-ssh</artifactId>-->
+                                <!--<version>2.8</version>-->
+                            <!--</dependency>-->
+                        <!--</dependencies>-->
+                        <!--<executions>-->
+                            <!--<execution>-->
+                                <!--<goals>-->
+                                    <!--<goal>deploy</goal>-->
+                                <!--</goals>-->
+                                <!--<phase>deploy</phase>-->
+                                <!--<configuration>-->
+                                    <!--<inputDirectory>${basedir}/target/site</inputDirectory>-->
+                                <!--</configuration>-->
+                            <!--</execution>-->
+                        <!--</executions>-->
+                    <!--</plugin>-->
                 </plugins>
             </build>
         </profile>


[45/50] incubator-ignite git commit: # sprint-4 - Added javadoc for the withAsync method.

Posted by sb...@apache.org.
# sprint-4 - Added javadoc for the withAsync method.


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

Branch: refs/heads/ignite-834
Commit: 2cb73176d3a671cb2ac784310a4d36b2d03bad29
Parents: 601cf09
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Apr 30 18:45:38 2015 -0700
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Apr 30 18:45:38 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2cb73176/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
index 76ebcee..f5b5153 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
@@ -22,7 +22,7 @@ package org.apache.ignite.lang;
  */
 public interface IgniteAsyncSupport {
     /**
-     * Gets component with asynchronous mode enabled.
+     * Gets instance of this component with asynchronous mode enabled.
      *
      * @return Component with asynchronous mode enabled.
      */


[05/50] incubator-ignite git commit: Merge branches 'ignite-776' and 'ignite-sprint-4' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-776

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


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

Branch: refs/heads/ignite-834
Commit: 8b18e879a2b5d9b087b92b7fdc4a0409ac0ac916
Parents: 895d95b 18b4c39
Author: Andrey <an...@gridgain.com>
Authored: Wed Apr 29 17:39:29 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Wed Apr 29 17:39:29 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               | 22 +++---
 .../processors/cache/IgniteCacheProxy.java      | 10 +--
 .../distributed/GridCacheLockAbstractTest.java  | 75 ++++++++++++++++++++
 .../DataStreamerMultiThreadedSelfTest.java      |  2 +
 4 files changed, 90 insertions(+), 19 deletions(-)
----------------------------------------------------------------------