You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2015/07/09 16:59:29 UTC

[01/28] incubator-ignite git commit: IGNITE-1026 - Always create client cache

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-964-1 bdf656704 -> 9413747ce


IGNITE-1026 - Always create client cache


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

Branch: refs/heads/ignite-964-1
Commit: eef2b372fe83f7626981f9ce1364d99da5d0c60a
Parents: af829d0
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 17 19:24:25 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 17 19:24:25 2015 -0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 71 +++++---------------
 1 file changed, 16 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/eef2b372/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 2f7f22c..e6a0994 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
@@ -1917,7 +1917,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                             req.clientStartOnly(true);
                     }
                     else
-                        return new GridFinishedFuture<>();
+                        req.clientStartOnly(true);
 
                     req.deploymentId(desc.deploymentId());
 
@@ -1948,14 +1948,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 ccfg = desc.cacheConfiguration();
 
             if (ccfg == null)
-                return new GridFinishedFuture<>(new CacheExistsException("Failed to start near cache " +
+                return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
                     "(a cache with the given name is not started): " + cacheName));
 
             if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
                     return new GridFinishedFuture<>();
                 else
-                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start near cache " +
+                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start client cache " +
                         "(local node is an affinity node for cache): " + cacheName));
             }
 
@@ -2523,12 +2523,17 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting cache for name: " + name);
 
-        IgniteCache<K, V> jcache = (IgniteCache<K, V>)jCacheProxies.get(maskNull(name));
+        String masked = maskNull(name);
 
-        if (jcache == null)
-            jcache = startJCache(name, true);
+        IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
+
+        if (cache == null) {
+            dynamicStartCache(null, name, null, false);
+
+            cache = jCacheProxies.get(masked);
+        }
 
-        return jcache == null ? null : ((IgniteCacheProxy<K, V>)jcache).internalProxy();
+        return cache == null ? null : (IgniteInternalCache<K, V>)cache.internalProxy();
     }
 
     /**
@@ -2638,57 +2643,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (desc != null && !desc.cacheType().userCache())
             throw new IllegalStateException("Failed to get cache because it is a system cache: " + cacheName);
 
-        if (cache == null)
-           cache = startJCache(cacheName, failIfNotStarted);
-
-        return (IgniteCacheProxy<K, V>)cache;
-    }
-
-    /**
-     * @param cacheName Cache name.
-     * @param failIfNotStarted If {@code true} throws {@link IllegalArgumentException} if cache is not started,
-     *        otherwise returns {@code null} in this case.
-     * @return Cache instance for given name.
-     * @throws IgniteCheckedException If failed.
-     */
-    private IgniteCacheProxy startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
-        checkEmptyTransactions();
-
-        String masked = maskNull(cacheName);
-
-        DynamicCacheDescriptor desc = registeredCaches.get(masked);
+        if (cache == null) {
+            dynamicStartCache(null, cacheName, null, false).get();
 
-        if (desc == null || desc.cancelled()) {
-            if (failIfNotStarted)
-                throw new IllegalArgumentException("Cache is not started: " + cacheName);
-
-            return null;
+            cache = jCacheProxies.get(masked);
         }
 
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
-
-        req.cacheName(cacheName);
-
-        req.deploymentId(desc.deploymentId());
-
-        CacheConfiguration cfg = new CacheConfiguration(desc.cacheConfiguration());
-
-        cfg.setNearConfiguration(null);
-
-        req.startCacheConfiguration(cfg);
-
-        req.cacheType(desc.cacheType());
-
-        req.clientStartOnly(true);
-
-        F.first(initiateCacheChanges(F.asList(req), false)).get();
-
-        IgniteCacheProxy cache = jCacheProxies.get(masked);
-
-        if (cache == null && failIfNotStarted)
-            throw new IllegalArgumentException("Cache is not started: " + cacheName);
-
-        return cache;
+        return (IgniteCacheProxy<K, V>)cache;
     }
 
     /**


[15/28] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by iv...@apache.org.
IGNITE-1026 - Count down latch 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/6f50ad9f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6f50ad9f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6f50ad9f

Branch: refs/heads/ignite-964-1
Commit: 6f50ad9f2d7a0e049863f235104c65c86ee8c5ad
Parents: 8ff3619
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 16:05:03 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 16:05:03 2015 -0700

----------------------------------------------------------------------
 .../continuous/CacheContinuousQueryHandler.java |  4 +--
 .../datastructures/DataStructuresProcessor.java | 31 +++++++++++++++++---
 .../GridCacheCountDownLatchImpl.java            | 12 ++++----
 3 files changed, 36 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index ff2905f..e059760 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -353,8 +353,8 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
         Iterable<CacheEntryEvent<? extends K, ? extends V>> evts = F.viewReadOnly(entries,
             new C1<CacheContinuousQueryEntry, CacheEntryEvent<? extends K, ? extends V>>() {
                 @Override public CacheEntryEvent<? extends K, ? extends V> apply(CacheContinuousQueryEntry e) {
-                    return new CacheContinuousQueryEvent<K, V>(cache, cctx, e);
-                };
+                    return new CacheContinuousQueryEvent<>(cache, cctx, e);
+                }
             }
         );
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 4d2ecbe..5c171e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1056,7 +1056,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         dsView.remove(key);
 
                         tx.commit();
-                    } else
+                    }
+                    else
                         tx.setRollbackOnly();
 
                     return null;
@@ -1147,19 +1148,41 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         GridCacheInternalKey key = evt.getKey();
 
                         // Notify latch on changes.
-                        GridCacheRemovable latch = dsMap.get(key);
+                        final GridCacheRemovable latch = dsMap.get(key);
 
                         GridCacheCountDownLatchValue val = (GridCacheCountDownLatchValue)val0;
 
                         if (latch instanceof GridCacheCountDownLatchEx) {
-                            GridCacheCountDownLatchEx latch0 = (GridCacheCountDownLatchEx)latch;
+                            final GridCacheCountDownLatchEx latch0 = (GridCacheCountDownLatchEx)latch;
 
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
                                 dsMap.remove(key);
 
-                                latch.onRemoved();
+                                IgniteInternalFuture<?> removeFut = ctx.closure().runLocalSafe(new GPR() {
+                                    @Override public void run() {
+                                        try {
+                                            removeCountDownLatch(latch0.name());
+                                        }
+                                        catch (IgniteCheckedException e) {
+                                            U.error(log, "Failed to remove count down latch: " + latch0.name(), e);
+                                        }
+                                    }
+                                });
+
+                                removeFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                                    @Override public void apply(IgniteInternalFuture<?> f) {
+                                        try {
+                                            f.get();
+                                        }
+                                        catch (IgniteCheckedException e) {
+                                            U.error(log, "Failed to remove count down latch: " + latch0.name(), e);
+                                        }
+
+                                        latch.onRemoved();
+                                    }
+                                });
                             }
                         }
                         else if (latch != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f50ad9f/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index a5353d8..2df6015 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -288,11 +288,13 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public void close() {
-        try {
-            ctx.kernalContext().dataStructures().removeCountDownLatch(name);
-        }
-        catch (IgniteCheckedException e) {
-            throw U.convertException(e);
+        if (!rmvd) {
+            try {
+                ctx.kernalContext().dataStructures().removeCountDownLatch(name);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
         }
     }
 


[23/28] incubator-ignite git commit: release notes

Posted by iv...@apache.org.
release notes


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

Branch: refs/heads/ignite-964-1
Commit: 0a569b8acfa4918dcd98a9fa0e4873f1e5b5737a
Parents: f13f594
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:55:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:55:58 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0a569b8a/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index ec8c4e6..0e22f1f 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -6,6 +6,7 @@ Apache Ignite In-Memory Data Fabric 1.3
 
 * Added auto-retries for cache operations in recoverable cases.
 * Fixed several issues with JTA integration.
+* Fixed several issues with Hibernate L2 cache.
 * Fixed issue with GAR files in source release.
 * Stability fixes for TCP discovery SPI.
 * Stability fixes for onheap and offheap SQL queries.


[03/28] incubator-ignite git commit: IGNITE-1026 - Fixing dynamic cache start methods.

Posted by iv...@apache.org.
IGNITE-1026 - Fixing dynamic cache start methods.


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

Branch: refs/heads/ignite-964-1
Commit: bb73b66c6d29de578da3aed2654c2a887c97337c
Parents: eef2b37
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 18 15:49:06 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 18 15:49:06 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++++-----
 .../processors/cache/GridCacheProcessor.java    | 31 ++++++++++++--------
 .../datastructures/DataStructuresProcessor.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +--
 4 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index e19d3d3..3ee260d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2265,7 +2265,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, true).get();
+            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, true, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2302,7 +2302,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         try {
             if (ctx.cache().cache(cacheCfg.getName()) == null)
-                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false).get();
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), null, false, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2325,7 +2325,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, true).get();
+            ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, true, true).get();
 
             return ctx.cache().publicJCache(cacheCfg.getName());
         }
@@ -2349,10 +2349,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             IgniteInternalCache<Object, Object> cache = ctx.cache().cache(cacheCfg.getName());
 
             if (cache == null)
-                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+                ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false, true).get();
             else {
                 if (cache.configuration().getNearConfiguration() == null)
-                    ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false).get();
+                    ctx.cache().dynamicStartCache(cacheCfg, cacheCfg.getName(), nearCfg, false, true).get();
             }
 
             return ctx.cache().publicJCache(cacheCfg.getName());
@@ -2372,7 +2372,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true).get();
+            ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true, true).get();
 
             IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
 
@@ -2399,10 +2399,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             IgniteInternalCache<Object, Object> internalCache = ctx.cache().cache(cacheName);
 
             if (internalCache == null)
-                ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+                ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false, true).get();
             else {
                 if (internalCache.configuration().getNearConfiguration() == null)
-                    ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
+                    ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false, true).get();
             }
 
             IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/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 e6a0994..ac2d7b1 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
@@ -1458,6 +1458,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (clientNodeStart && !affNodeStart) {
                 if (nearCfg != null)
                     ccfg.setNearConfiguration(nearCfg);
+                else
+                    ccfg.setNearConfiguration(null);
             }
 
             CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
@@ -1746,7 +1748,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> createFromTemplate(String cacheName) {
         CacheConfiguration cfg = createConfigFromTemplate(cacheName);
 
-        return dynamicStartCache(cfg, cacheName, null, true);
+        return dynamicStartCache(cfg, cacheName, null, true, true);
     }
 
     /**
@@ -1762,7 +1764,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheConfiguration cfg = createConfigFromTemplate(cacheName);
 
-            return dynamicStartCache(cfg, cacheName, null, false);
+            return dynamicStartCache(cfg, cacheName, null, false, true);
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(e);
@@ -1855,9 +1857,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         @Nullable CacheConfiguration ccfg,
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
-        boolean failIfExists
+        boolean failIfExists,
+        boolean failIfNotStarted
     ) {
-        return dynamicStartCache(ccfg, cacheName, nearCfg, CacheType.USER, failIfExists);
+        return dynamicStartCache(ccfg, cacheName, nearCfg, CacheType.USER, failIfExists, failIfNotStarted);
     }
 
     /**
@@ -1875,12 +1878,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         String cacheName,
         @Nullable NearCacheConfiguration nearCfg,
         CacheType cacheType,
-        boolean failIfExists
+        boolean failIfExists,
+        boolean failIfNotStarted
     ) {
         checkEmptyTransactions();
 
-        assert ccfg != null || nearCfg != null;
-
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
 
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
@@ -1947,9 +1949,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (desc != null && !desc.cancelled())
                 ccfg = desc.cacheConfiguration();
 
-            if (ccfg == null)
-                return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
-                    "(a cache with the given name is not started): " + cacheName));
+            if (ccfg == null) {
+                if (failIfNotStarted)
+                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
+                        "(a cache with the given name is not started): " + cacheName));
+                else
+                    return new GridFinishedFuture<>();
+            }
 
             if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
@@ -1961,6 +1967,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             req.deploymentId(desc.deploymentId());
             req.startCacheConfiguration(ccfg);
+
         }
 
         if (nearCfg != null)
@@ -2528,7 +2535,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         if (cache == null) {
-            dynamicStartCache(null, name, null, false);
+            dynamicStartCache(null, name, null, false, false);
 
             cache = jCacheProxies.get(masked);
         }
@@ -2644,7 +2651,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to get cache because it is a system cache: " + cacheName);
 
         if (cache == null) {
-            dynamicStartCache(null, cacheName, null, false).get();
+            dynamicStartCache(null, cacheName, null, false, failIfNotStarted).get();
 
             cache = jCacheProxies.get(masked);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 473a2ac..dcd22cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -819,7 +819,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         CacheConfiguration newCfg = cacheConfiguration(cfg, cacheName);
 
-        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false).get();
+        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
 
         return cacheName;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb73b66c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index 7905565..7d00417 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -143,7 +143,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 ccfg.setName(DYNAMIC_CACHE_NAME);
 
-                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true, true));
 
                 return null;
             }
@@ -203,7 +203,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
                 IgniteEx kernal = grid(ThreadLocalRandom.current().nextInt(nodeCount()));
 
-                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true));
+                futs.add(kernal.context().cache().dynamicStartCache(ccfg, ccfg.getName(), null, true, true));
 
                 return null;
             }


[24/28] incubator-ignite git commit: Merge branches 'ignite-1026' and 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026

Posted by iv...@apache.org.
Merge branches 'ignite-1026' and 'master' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026


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

Branch: refs/heads/ignite-964-1
Commit: c134dcfa5e7bb5dbc7a533f3d047e6e40cf2ce4e
Parents: 3089ace 0a569b8
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 8 12:20:13 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 8 12:20:13 2015 -0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 ++
 dev-tools/slurp.sh                              |   2 +-
 modules/core/pom.xml                            |   4 +-
 .../cache/eviction/fifo/FifoEvictionPolicy.java |   5 -
 .../cache/eviction/lru/LruEvictionPolicy.java   |   5 -
 .../eviction/sorted/SortedEvictionPolicy.java   |  19 +-
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../apache/ignite/internal/IgniteKernal.java    |  10 +-
 .../internal/interop/InteropIgnition.java       |  48 +++--
 .../internal/interop/InteropProcessor.java      |   7 +
 .../processors/cache/CacheObjectImpl.java       |   1 -
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../dht/GridPartitionedGetFuture.java           |  13 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |  35 +++-
 .../processors/query/GridQueryProcessor.java    |   5 +
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  46 +++++
 .../tcp/internal/TcpDiscoveryNode.java          |   2 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |   8 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |  10 +-
 .../cache/CacheFutureExceptionSelfTest.java     | 158 +++++++++++++++
 .../GridCachePartitionedNodeRestartTest.java    |   5 -
 ...ePartitionedOptimisticTxNodeRestartTest.java |   2 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 -
 .../GridCacheReplicatedNodeRestartSelfTest.java |   5 -
 ...acheAtomicReplicatedNodeRestartSelfTest.java |  14 +-
 ...heConcurrentEvictionConsistencySelfTest.java |  15 +-
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  38 ++++
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |  80 ++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   5 +
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../CacheAbstractQueryMetricsSelfTest.java      | 157 +++++++++++++-
 .../cache/CacheLocalQueryMetricsSelfTest.java   |  33 +++
 ...titionedQueryMetricsDistributedSelfTest.java |  33 +++
 ...chePartitionedQueryMetricsLocalSelfTest.java |  33 +++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 ---
 ...plicatedQueryMetricsDistributedSelfTest.java |  33 +++
 ...acheReplicatedQueryMetricsLocalSelfTest.java |  33 +++
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 ---
 .../IgniteCacheQuerySelfTestSuite.java          |   7 +-
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 modules/mesos/pom.xml                           |   1 -
 modules/rest-http/pom.xml                       |  14 +-
 modules/urideploy/pom.xml                       |   8 +-
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 modules/web/pom.xml                             |   6 +-
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 modules/yardstick/config/ignite-base-config.xml |  23 +++
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |  42 ++++
 .../ignite/yardstick/cache/model/Person1.java   |  55 +++++
 .../ignite/yardstick/cache/model/Person2.java   |  67 ++++++
 .../ignite/yardstick/cache/model/Person8.java   | 109 ++++++++++
 parent/pom.xml                                  |   1 +
 scripts/git-patch-prop.sh                       |   2 +-
 75 files changed, 1695 insertions(+), 271 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c134dcfa/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c134dcfa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[25/28] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-964-1

Posted by iv...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-964-1


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

Branch: refs/heads/ignite-964-1
Commit: 7763a37b1d02bc4136dad8e0be1e481a7edfd573
Parents: bdf6567 c134dcf
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 9 15:57:54 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 9 15:57:54 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 ++
 .../src/main/java/org/apache/ignite/Ignite.java |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  32 +++---
 .../processors/cache/GridCacheProcessor.java    |  97 +++++------------
 .../continuous/CacheContinuousQueryHandler.java |   4 +-
 .../datastructures/DataStructuresProcessor.java |  39 +++++--
 .../GridCacheCountDownLatchImpl.java            |  15 ++-
 ...cheStoreSessionListenerAbstractSelfTest.java |   1 -
 .../IgniteCacheConfigurationTemplateTest.java   |  26 +----
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  16 +--
 .../IgniteDynamicClientCacheStartSelfTest.java  |   5 +-
 .../IgniteClientDataStructuresAbstractTest.java | 109 ++++++++++++++-----
 .../IgniteCountDownLatchAbstractSelfTest.java   |  12 +-
 13 files changed, 200 insertions(+), 170 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7763a37b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------


[08/28] incubator-ignite git commit: IGNITE-1026 - Dynamic cache start fix

Posted by iv...@apache.org.
IGNITE-1026 - Dynamic cache start 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/08ea4ccf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/08ea4ccf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/08ea4ccf

Branch: refs/heads/ignite-964-1
Commit: 08ea4ccf251afc9d485c5409c4b43539d01c62fb
Parents: 7cf9d76
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 15:30:38 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 15:30:38 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheProcessor.java       | 9 ---------
 .../store/CacheStoreSessionListenerAbstractSelfTest.java    | 1 -
 2 files changed, 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08ea4ccf/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 4f1a3e6..0f9247f 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
@@ -1957,17 +1957,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     return new GridFinishedFuture<>();
             }
 
-            if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
-                if (ccfg.getNearConfiguration() != null)
-                    return new GridFinishedFuture<>();
-                else
-                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start client cache " +
-                        "(local node is an affinity node for cache): " + cacheName));
-            }
-
             req.deploymentId(desc.deploymentId());
             req.startCacheConfiguration(ccfg);
-
         }
 
         if (nearCfg != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/08ea4ccf/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
index adac0b2..0634197 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -145,7 +145,6 @@ public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridComm
         assertEquals(1, writeCnt.get());
         assertEquals(1, deleteCnt.get());
         assertEquals(0, reuseCnt.get());
-
     }
 
     /**


[13/28] incubator-ignite git commit: Merge branch 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026

Posted by iv...@apache.org.
Merge branch 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026


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

Branch: refs/heads/ignite-964-1
Commit: 378e344aba5048e36d86ca0944435586d45b7032
Parents: 7c935a5 d3783a1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 1 15:26:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 1 15:26:40 2015 -0700

----------------------------------------------------------------------
 assembly/dependencies-fabric.xml                |   1 +
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../ClientAbstractConnectivitySelfTest.java     |   4 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../org/apache/ignite/cluster/ClusterGroup.java |  18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   2 +
 .../configuration/CacheConfiguration.java       | 105 +--
 .../configuration/NearCacheConfiguration.java   |  10 +-
 .../ignite/internal/GridKernalContextImpl.java  |   5 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  50 +-
 .../cluster/IgniteClusterAsyncImpl.java         |  12 +-
 .../managers/communication/GridIoManager.java   |  49 +-
 .../discovery/GridDiscoveryManager.java         |  32 +-
 .../cache/GridCacheDeploymentManager.java       |  10 +-
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../processors/cache/GridCacheProcessor.java    |  62 +-
 .../processors/cache/IgniteCacheFutureImpl.java |  42 +
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../processors/clock/GridClockServer.java       |  21 +-
 .../processors/plugin/CachePluginManager.java   |  10 +-
 .../processors/rest/GridRestProcessor.java      |   4 +-
 .../handlers/task/GridTaskCommandHandler.java   |  12 +-
 .../processors/task/GridTaskWorker.java         |   4 +-
 .../internal/util/GridConfigurationFinder.java  |  55 +-
 .../ignite/internal/util/IgniteUtils.java       |   6 +-
 .../internal/util/future/IgniteFutureImpl.java  |  18 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  10 +-
 .../util/nio/GridNioMessageTracker.java         |  23 +-
 .../apache/ignite/internal/visor/VisorJob.java  |   2 +
 .../internal/visor/log/VisorLogSearchTask.java  |   2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |   4 +
 .../visor/query/VisorQueryCleanupTask.java      |  14 +
 .../util/VisorClusterGroupEmptyException.java   |  37 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 151 ++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 105 ++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   3 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  74 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../core/src/test/config/spark/spark-config.xml |  46 ++
 modules/core/src/test/config/tests.properties   |   6 +-
 .../internal/ClusterGroupAbstractTest.java      | 777 ++++++++++++++++++
 .../internal/ClusterGroupHostsSelfTest.java     | 141 ++++
 .../ignite/internal/ClusterGroupSelfTest.java   | 251 ++++++
 .../internal/GridDiscoveryEventSelfTest.java    |  12 +-
 .../internal/GridProjectionAbstractTest.java    | 784 -------------------
 .../ignite/internal/GridProjectionSelfTest.java | 251 ------
 .../apache/ignite/internal/GridSelfTest.java    |   2 +-
 .../IgniteTopologyPrintFormatSelfTest.java      | 289 +++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ---
 .../IgniteDaemonNodeMarshallerCacheTest.java    | 192 +++++
 ...achePartitionedMultiNodeFullApiSelfTest.java |   4 +-
 .../internal/util/IgniteUtilsSelfTest.java      |  22 +
 .../GridP2PContinuousDeploymentSelfTest.java    |   2 -
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 265 ++++++-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   7 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |   1 +
 modules/core/src/test/resources/helloworld.gar  | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/helloworld1.gar | Bin 6092 -> 0 bytes
 modules/core/src/test/resources/readme.txt      |   6 -
 modules/docker/Dockerfile                       |  55 ++
 modules/docker/README.txt                       |  11 +
 modules/docker/build_users_libs.sh              |  39 +
 modules/docker/download_ignite.sh               |  49 ++
 modules/docker/execute.sh                       |  62 ++
 modules/docker/run.sh                           |  34 +
 modules/extdata/p2p/pom.xml                     |   4 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |   2 +-
 modules/extdata/uri/META-INF/ignite.xml         |  38 +
 .../extdata/uri/modules/uri-dependency/pom.xml  |  42 +
 .../deployment/uri/tasks/GarHelloWorldBean.java |  60 ++
 .../src/main/resources/gar-example.properties   |  18 +
 modules/extdata/uri/pom.xml                     |  62 +-
 .../deployment/uri/tasks/GarHelloWorldTask.java |  81 ++
 .../deployment/uri/tasks/gar-spring-bean.xml    |  29 +
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 .../ignite/schema/model/PojoDescriptor.java     |   2 +
 .../apache/ignite/schema/model/PojoField.java   |   1 +
 .../parser/dialect/OracleMetadataDialect.java   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 .../org/apache/ignite/spark/IgniteContext.scala |  50 +-
 .../org/apache/ignite/spark/IgniteRddSpec.scala |  18 +
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |  16 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |  13 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../commands/cache/VisorCacheCommand.scala      |   7 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  14 +-
 scripts/git-patch-prop.sh                       |   2 +-
 116 files changed, 3424 insertions(+), 1498 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/378e344a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[11/28] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by iv...@apache.org.
IGNITE-1026 - Count down latch 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/389bad8d
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/389bad8d
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/389bad8d

Branch: refs/heads/ignite-964-1
Commit: 389bad8d1cac892b01c0a5e5e9479337410c1109
Parents: fe92bfe
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 24 18:47:42 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 24 18:47:42 2015 -0700

----------------------------------------------------------------------
 .../processors/datastructures/DataStructuresProcessor.java  | 9 ---------
 .../datastructures/GridCacheCountDownLatchImpl.java         | 3 ---
 2 files changed, 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/389bad8d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index c26e14d..4d2ecbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.transactions.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
@@ -1158,14 +1157,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
-                                try {
-                                    removeCountDownLatch(latch0.name());
-                                }
-                                catch (IgniteCheckedException e) {
-                                    U.error(log, "Failed to automatically delete count down latch: " +
-                                        latch0.name(), e);
-                                }
-
                                 dsMap.remove(key);
 
                                 latch.onRemoved();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/389bad8d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 33547d9..a5353d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -288,9 +288,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public void close() {
-        if (rmvd)
-            return;
-
         try {
             ctx.kernalContext().dataStructures().removeCountDownLatch(name);
         }


[12/28] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by iv...@apache.org.
IGNITE-1026 - Count down latch 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/7c935a58
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/7c935a58
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/7c935a58

Branch: refs/heads/ignite-964-1
Commit: 7c935a58ab6cc37ac062f1c37a2650504e9ef95c
Parents: 389bad8
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 26 14:30:16 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 26 14:30:16 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c935a58/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 6c7c511..60ffce0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2421,9 +2421,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /**
      * @param cache Cache.
      */
-    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) {
+    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) throws IgniteCheckedException {
         if (!cache.context().isNear())
-            throw new IgniteException("Failed to start near cache " +
+            throw new IgniteCheckedException("Failed to start near cache " +
                 "(a cache with the same name without near cache is already started)");
     }
 


[20/28] incubator-ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-964-1
Commit: fa5cb918dcb0fe4253332fd9be859803ad014ddb
Parents: 459d702 4c9d8c2
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:39:36 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:39:36 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |   4 +
 .../configuration/TransactionConfiguration.java |  23 +++
 .../processors/cache/GridCacheAttributes.java   |   3 +
 .../processors/cache/GridCacheContext.java      |   8 +-
 .../processors/cache/GridCacheIoManager.java    |   8 +-
 .../processors/cache/GridCacheProcessor.java    |  21 +-
 .../cache/GridCacheSharedContext.java           |  15 +-
 .../distributed/near/GridNearGetFuture.java     |   4 +-
 .../cache/jta/CacheJtaManagerAdapter.java       |  17 +-
 .../cache/jta/CacheNoopJtaManager.java          |   2 +-
 .../visor/cache/VisorCacheConfiguration.java    |  11 -
 .../cache/CacheFutureExceptionSelfTest.java     | 161 +++++++--------
 .../loadtests/hashmap/GridCacheTestContext.java |   4 +-
 .../HibernateTransactionalDataRegion.java       |  12 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   7 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |   5 -
 .../apache/ignite/cache/jta/CacheTmLookup.java  |   3 +-
 .../processors/cache/jta/CacheJtaManager.java   |  72 ++++++-
 .../cache/jta/GridCacheXAResource.java          |  16 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |  52 +++--
 .../GridTmLookupLifecycleAwareSelfTest.java     |  29 ++-
 modules/kafka/licenses/apache-2.0.txt           | 202 +++++++++++++++++++
 modules/kafka/pom.xml                           |  11 -
 .../commands/cache/VisorCacheCommand.scala      |   2 -
 .../config/benchmark-index-put.properties       |  64 ------
 .../config/benchmark-put-indexed-val.properties |  64 ++++++
 .../cache/IgnitePutIndex1Benchmark.java         |  42 ----
 .../cache/IgnitePutIndex2Benchmark.java         |  42 ----
 .../cache/IgnitePutIndexedValue1Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue2Benchmark.java  |  42 ++++
 .../cache/IgnitePutIndexedValue8Benchmark.java  |   2 +-
 .../ignite/yardstick/cache/model/Person1.java   |  29 +--
 .../ignite/yardstick/cache/model/Person2.java   |  45 +----
 .../ignite/yardstick/cache/model/Person8.java   | 155 +-------------
 34 files changed, 661 insertions(+), 558 deletions(-)
----------------------------------------------------------------------



[27/28] incubator-ignite git commit: #ignite-964: change cache-put-get-example.js

Posted by iv...@apache.org.
#ignite-964: change cache-put-get-example.js


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

Branch: refs/heads/ignite-964-1
Commit: bfc899e42dcb652412e9a203dd46295ff6f5affa
Parents: 242c21b
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 9 17:40:03 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 9 17:40:03 2015 +0300

----------------------------------------------------------------------
 examples/src/main/js/cache-api-example.js     |   2 +-
 examples/src/main/js/cache-put-get-example.js | 132 +++++++++------------
 2 files changed, 59 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bfc899e4/examples/src/main/js/cache-api-example.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/cache-api-example.js b/examples/src/main/js/cache-api-example.js
index d17276a..1514941 100644
--- a/examples/src/main/js/cache-api-example.js
+++ b/examples/src/main/js/cache-api-example.js
@@ -55,7 +55,7 @@ function main() {
             cache.getAndPut(1, "1", onGetAndPut)
         });
 
-        onGetAndPut = function(err, entry) {
+        function onGetAndPut(err, entry) {
             console.log(">>> Get and put finished [result=" + entry + "]");
 
             // Put and do not return previous value.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bfc899e4/examples/src/main/js/cache-put-get-example.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/cache-put-get-example.js b/examples/src/main/js/cache-put-get-example.js
index 7a9b035..75da096 100644
--- a/examples/src/main/js/cache-put-get-example.js
+++ b/examples/src/main/js/cache-put-get-example.js
@@ -16,104 +16,88 @@
  */
 
 var apacheIgnite = require("apache-ignite");
-
 var Ignition = apacheIgnite.Ignition;
 var CacheEntry = apacheIgnite.CacheEntry;
 
-Ignition.start(['127.0.0.1:9095'], null, onConnect);
-
-function onConnect(err, ignite) {
-    if (err)
-        throw err;
-
-   ignite.getOrCreateCache("PutGetExampleCache", function(err, cache) {
-            putGet(cache);
-
-            putAllGetAll(cache);
-        });
-}
-
-putGet = function(cache) {
-    console.log(">>> Cache put-get example started.");
-
-    var keyCnt = 20;
-
-    var putCnt = 0;
-
-    var onGet = function(err, res) {
-        if (err) {
-            console.log("Error: " + err);
-
-            throw new Error(err);
-        }
-
-        console.log("Get val=" + res);
+/**
+  * This example demonstrates very basic operations on cache, such as 'put' and 'get'.
+  * <p>
+  * Remote nodes should always be started with special configuration file which
+  * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/js/example-js-cache.xml'}.
+  * <p>
+  * Alternatively you can run {@link ExampleJsNodeStartup} in another JVM which will
+  * start node with {@code examples/config/js/example-js-cache.xml} configuration.
+  */
+function main() {
+    /** Cache name. */
+    var cacheName = "PutGetExampleCache";
+
+    /** Connect to node that started with {@code examples/config/js/example-js-cache.xml} configuration. */
+    Ignition.start(['127.0.0.1:9095'], null, onConnect);
+
+    function onConnect(err, ignite) {
+       ignite.getOrCreateCache(cacheName, function(err, cache) { putGetExample(ignite, cache); });
     }
 
-    var onPut = function(err) {
-        if (err) {
-            console.log("Error: " + err);
+    /** Execute individual puts and gets. */
+    putGetExample = function(ignite, cache) {
+        console.log(">>> Cache put-get example started.");
 
-            throw new Error(err);
-        }
+        var key = 1;
+
+        // Store key in cache.
+        cache.put(key, "1", onPut);
 
-        if (putCnt < keyCnt - 1) {
-            putCnt++;
+        function onPut(err) {
+            console.log(">>> Stored values in cache.");
 
-            return;
+            cache.get(key, onGet);
         }
 
-        console.log(">>> Stored values in cache.");
+        function onGet(err, res) {
+            console.log("Get val=" + res);
 
-        for (var i = 0; i < keyCnt; i++) {
-            cache.get(i, onGet);
+            putAllGetAll(ignite, cache);
         }
     }
 
-    // Store keys in cache.
-    for (var i = 0; i < keyCnt; i++) {
-        cache.put(i, i.toString(), onPut);
-    }
-}
-
-putAllGetAll = function(cache) {
-    console.log(">>> Starting putAll-getAll example.");
-
-    var keyCnt = 20;
+    /** Execute bulk {@code putAll(...)} and {@code getAll(...)} operations. */
+    function putAllGetAll(ignite, cache) {
+        console.log(">>> Starting putAll-getAll example.");
 
-    var batch = [];
-    var keys = [];
+        var keyCnt = 20;
 
-    for (var i = keyCnt; i < keyCnt + keyCnt; ++i) {
-        var key = i;
+        // Create batch.
+        var batch = [];
+        var keys = [];
 
-        var val = "bulk-" + i;
+        for (var i = keyCnt; i < keyCnt + keyCnt; ++i) {
+            var key = i;
+            var val = "bulk-" + i;
 
-        keys.push(key);
-        batch.push(new CacheEntry(key, val));
-    }
+            keys.push(key);
+            batch.push(new CacheEntry(key, val));
+        }
 
-    var onGetAll = function(err, entries) {
-        if (err) {
-            console.log("Error: " + err);
+        cache.putAll(batch, onPutAll);
 
-            throw new Error(err);
-        }
+        function onPutAll(err) {
+            console.log(">>> Stored values in cache.");
 
-        for (var e of entries) {
-            console.log("Got entry [key=" + e.key + ", val=" + e.value + ']');
+            cache.getAll(keys, onGetAll);
         }
-    }
 
-    var onPutAll= function(err) {
-        if (err) {
-            console.log("Error: " + err);
+        function onGetAll(err, entries) {
+            for (var e of entries) {
+                console.log("Got entry [key=" + e.key + ", val=" + e.value + ']');
+            }
 
-            throw new Error(err);
+            // Destroying cache.
+            ignite.destroyCache(cacheName, function(err) {
+                    console.log(">>> End of cache put-get example.");
+                });
         }
-
-        cache.getAll(keys, onGetAll);
     }
+}
 
-    cache.putAll(batch, onPutAll);
-}
\ No newline at end of file
+main();
\ No newline at end of file


[05/28] incubator-ignite git commit: IGNITE-1026 - Data structures fix

Posted by iv...@apache.org.
IGNITE-1026 - Data structures 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/1419d39b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1419d39b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1419d39b

Branch: refs/heads/ignite-964-1
Commit: 1419d39ba80da9f85b28c708589f74dcec6514d5
Parents: 2f089f1
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:54:51 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:54:51 2015 -0700

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheProcessor.java     | 2 +-
 .../processors/datastructures/DataStructuresProcessor.java       | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/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 ac2d7b1..4f1a3e6 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
@@ -2535,7 +2535,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         if (cache == null) {
-            dynamicStartCache(null, name, null, false, false);
+            dynamicStartCache(null, name, null, false, true).get();
 
             cache = jCacheProxies.get(masked);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1419d39b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index dcd22cd..26e2f6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -177,7 +177,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         if (initLatch.getCount() > 0) {
             initFailed = true;
-            
+
             initLatch.countDown();
         }
 
@@ -890,7 +890,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
             String cacheName = ((CollectionInfo)oldInfo.info).cacheName;
 
-            GridCacheContext cacheCtx = ctx.cache().internalCache(cacheName).context();
+            GridCacheContext cacheCtx = ctx.cache().getOrStartCache(cacheName).context();
 
             return c.applyx(cacheCtx);
         }


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

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


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

Branch: refs/heads/ignite-964-1
Commit: fe92bfec6ee1eaf55df5d1654f51b9b143518a09
Parents: 9adc180 b29ff1c
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jun 24 15:49:23 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jun 24 15:49:23 2015 -0700

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../configuration/IgniteReflectionFactory.java  |  81 +++-
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheIoManager.java    |  64 ++-
 .../GridCachePartitionExchangeManager.java      |  70 ++-
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   9 +-
 .../GridDhtPartitionsExchangeFuture.java        |  95 +++-
 .../datastructures/DataStructuresProcessor.java |  64 +--
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../processors/task/GridTaskProcessor.java      |  23 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  61 ++-
 .../tcp/TcpCommunicationSpiMBean.java           |   8 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  68 ++-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../GridTaskFailoverAffinityRunTest.java        | 170 +++++++
 .../CacheReadThroughAtomicRestartSelfTest.java  |  32 ++
 ...heReadThroughLocalAtomicRestartSelfTest.java |  32 ++
 .../CacheReadThroughLocalRestartSelfTest.java   |  32 ++
 ...dThroughReplicatedAtomicRestartSelfTest.java |  32 ++
 ...cheReadThroughReplicatedRestartSelfTest.java |  32 ++
 .../cache/CacheReadThroughRestartSelfTest.java  | 133 ++++++
 .../GridCacheAbstractFailoverSelfTest.java      |   6 +-
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++++++++++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  37 ++
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   4 +-
 .../GridCacheSetAbstractSelfTest.java           |  22 +-
 .../IgniteDataStructureWithJobTest.java         | 111 +++++
 ...ridCachePartitionNotLoadedEventSelfTest.java |  82 ++++
 .../distributed/IgniteCache150ClientsTest.java  | 189 ++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../distributed/IgniteCacheManyClientsTest.java |   2 +
 .../IgniteCacheTxMessageRecoveryTest.java       |   5 +
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   5 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |  49 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 +
 .../IgniteCacheTxStoreSessionTest.java          |   4 +
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   1 -
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   8 +
 .../testsuites/IgniteClientTestSuite.java       |  38 ++
 .../testsuites/IgniteComputeGridTestSuite.java  |   1 +
 .../ignite/util/TestTcpCommunicationSpi.java    |  21 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |  80 +---
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 ++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +-
 .../internal/processors/hadoop/HadoopUtils.java | 237 ----------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 ++++++++++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 +++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 ++++++++
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 +
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../IgniteWebSessionSelfTestSuite.java          |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 117 files changed, 2759 insertions(+), 683 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe92bfec/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/fe92bfec/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------


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

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


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

Branch: refs/heads/ignite-964-1
Commit: 7cf9d76b8323162ecae69b26cc3d5fd12320b7a3
Parents: 431a62e 415264e
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 14:37:41 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 14:37:41 2015 -0700

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  71 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 .../core/src/main/resources/ignite.properties   |   2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 ++++-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 59 files changed, 545 insertions(+), 73 deletions(-)
----------------------------------------------------------------------



[22/28] incubator-ignite git commit: release notes

Posted by iv...@apache.org.
release notes


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

Branch: refs/heads/ignite-964-1
Commit: f13f5946beaae7115f2ca5796988624ac36c193a
Parents: 064d079
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:52:13 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:52:13 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f13f5946/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index bcfed27..ec8c4e6 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -6,6 +6,7 @@ Apache Ignite In-Memory Data Fabric 1.3
 
 * Added auto-retries for cache operations in recoverable cases.
 * Fixed several issues with JTA integration.
+* Fixed issue with GAR files in source release.
 * Stability fixes for TCP discovery SPI.
 * Stability fixes for onheap and offheap SQL queries.
 * Bug fixes in In-Memory Accelerator For Apache Hadoop.


[28/28] incubator-ignite git commit: #ignite-964: change run-cache-script.js

Posted by iv...@apache.org.
#ignite-964: change run-cache-script.js


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

Branch: refs/heads/ignite-964-1
Commit: 9413747ce03999b5e746c4387eae207f75268799
Parents: bfc899e
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 9 17:59:17 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 9 17:59:17 2015 +0300

----------------------------------------------------------------------
 examples/src/main/js/cache-put-get-example.js   | 10 +--
 .../main/js/compute-callable-cache-example.js   | 49 -------------
 examples/src/main/js/run-cache-script.js        | 76 ++++++++++++++++++++
 3 files changed, 81 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9413747c/examples/src/main/js/cache-put-get-example.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/cache-put-get-example.js b/examples/src/main/js/cache-put-get-example.js
index 75da096..80c2080 100644
--- a/examples/src/main/js/cache-put-get-example.js
+++ b/examples/src/main/js/cache-put-get-example.js
@@ -55,7 +55,7 @@ function main() {
         }
 
         function onGet(err, res) {
-            console.log("Get val=" + res);
+            console.log("Get value=" + res);
 
             putAllGetAll(ignite, cache);
         }
@@ -79,23 +79,23 @@ function main() {
             batch.push(new CacheEntry(key, val));
         }
 
+        // Bulk-store entries in cache.
         cache.putAll(batch, onPutAll);
 
         function onPutAll(err) {
             console.log(">>> Stored values in cache.");
 
+            // Bulk-get values from cache.
             cache.getAll(keys, onGetAll);
         }
 
         function onGetAll(err, entries) {
             for (var e of entries) {
-                console.log("Got entry [key=" + e.key + ", val=" + e.value + ']');
+                console.log("Got entry [key=" + e.key + ", value=" + e.value + ']');
             }
 
             // Destroying cache.
-            ignite.destroyCache(cacheName, function(err) {
-                    console.log(">>> End of cache put-get example.");
-                });
+            ignite.destroyCache(cacheName, function(err) { console.log(">>> End of cache put-get example."); });
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9413747c/examples/src/main/js/compute-callable-cache-example.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/compute-callable-cache-example.js b/examples/src/main/js/compute-callable-cache-example.js
deleted file mode 100644
index 1b92d7c..0000000
--- a/examples/src/main/js/compute-callable-cache-example.js
+++ /dev/null
@@ -1,49 +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.
- */
-
-var apacheIgnite = require("apache-ignite");
-var Ignition = apacheIgnite.Ignition;
-
-var cacheName = "ComputeCallableCacheExample";
-
-Ignition.start(['127.0.0.1:9095'], null, onConnect);
-
-function onConnect(err, ignite) {
-    console.log(">>> Compute callable example started.");
-
-    var f = function (args) {
-        print(">>> Hello node: " + ignite.name());
-
-        var cache = ignite.getOrCreateCache(args);
-
-        cache.put(ignite.name(), "Hello");
-
-        return ignite.name();
-    }
-
-    var onRunScript = function(err, igniteName) {
-        var cache = ignite.cache(cacheName);
-
-        cache.get(igniteName, function(err, res) {
-                console.log(res+ " " + igniteName);
-
-                console.log(">>> Check all nodes for output (this node is also part of the cluster).");
-            });
-    }
-
-    ignite.compute().runScript(f, cacheName, onRunScript);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9413747c/examples/src/main/js/run-cache-script.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/run-cache-script.js b/examples/src/main/js/run-cache-script.js
new file mode 100644
index 0000000..1640cea
--- /dev/null
+++ b/examples/src/main/js/run-cache-script.js
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+var apacheIgnite = require("apache-ignite");
+var Ignition = apacheIgnite.Ignition;
+
+/**
+  * This example demonstrates very basic operations on cache in functions for Compute.run.
+  * <p>
+  * Remote nodes should always be started with special configuration file which
+  * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/js/example-js-cache.xml'}.
+  * <p>
+  * Alternatively you can run {@link ExampleJsNodeStartup} in another JVM which will
+  * start node with {@code examples/config/js/example-js-cache.xml} configuration.
+  */
+function main() {
+    /** Cache name. */
+    var cacheName = "RunCacheScriptCache";
+
+    /** Connect to node that started with {@code examples/config/js/example-js-cache.xml} configuration. */
+    Ignition.start(['127.0.0.1:9095'], null, onConnect);
+
+    function onConnect(err, ignite) {
+        console.log(">>> Run cache script example started.");
+
+        ignite.getOrCreateCache(cacheName, function(err, cache) { runCacheScript(ignite, cache); });
+    }
+
+    function runCacheScript(ignite, cache) {
+        var key = "John";
+        var person = {"firstName": "John", "lastName": "Doe", "salary" : 2000};
+
+        // Store person in the cache
+        cache.put(key, person, onPut);
+
+        function onPut(err) {
+            var job = function (args) {
+                print(">>> Hello node: " + ignite.name());
+
+                var cacheName = args[0];
+                var key = args[1];
+
+                /** Get cache with name. */
+                var cache = ignite.cache(cacheName);
+
+                /** Get person with name John. */
+                var val = cache.get(key);
+
+                return val.salary;
+            }
+
+            var onRunScript = function(err, salary) {
+               console.log(">>> " + key + "'s salary is " + salary);
+            }
+
+            /** Run remote job on server ignite node with arguments [cacheName, key]. */
+            ignite.compute().runScript(job, [cacheName, key], onRunScript);
+        }
+    }
+}
+
+main();
\ No newline at end of file


[06/28] incubator-ignite git commit: IGNITE-1026 - Data structures fix

Posted by iv...@apache.org.
IGNITE-1026 - Data structures 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/431a62e9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/431a62e9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/431a62e9

Branch: refs/heads/ignite-964-1
Commit: 431a62e99d0e853d2e4cb412d6599cbfe3407fe1
Parents: 1419d39
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 18:11:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 18:11:21 2015 -0700

----------------------------------------------------------------------
 .../datastructures/DataStructuresProcessor.java |  11 +-
 .../IgniteClientDataStructuresAbstractTest.java | 109 ++++++++++++++-----
 2 files changed, 93 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/431a62e9/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 26e2f6c..f299a69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -819,7 +819,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         CacheConfiguration newCfg = cacheConfiguration(cfg, cacheName);
 
-        ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
+        if (ctx.cache().cache(cacheName) == null)
+            ctx.cache().dynamicStartCache(newCfg, cacheName, null, CacheType.INTERNAL, false, true).get();
 
         return cacheName;
     }
@@ -1179,6 +1180,14 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
+                                try {
+                                    removeCountDownLatch(latch0.name());
+                                }
+                                catch (IgniteCheckedException e) {
+                                    U.error(log, "Failed to automatically delete count down latch: " +
+                                        latch0.name(), e);
+                                }
+
                                 dsMap.remove(key);
 
                                 latch.onRemoved();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/431a62e9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 5a6be8e..bcfb713 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -77,12 +77,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testSequence() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+        testSequence(clientNode, srvNode);
+        testSequence(srvNode, clientNode);
+    }
 
-        try (IgniteAtomicSequence seq = clientNode.atomicSequence("seq1", 1L, true)) {
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testSequence(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.atomicSequence("seq1", 1L, false));
+        assertNull(other.atomicSequence("seq1", 1L, false));
+
+        try (IgniteAtomicSequence seq = creator.atomicSequence("seq1", 1L, true)) {
             assertNotNull(seq);
 
             assertEquals(1L, seq.get());
@@ -91,13 +101,13 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertEquals(2L, seq.get());
 
-            IgniteAtomicSequence seq0 = srvNode.atomicSequence("seq1", 1L, false);
+            IgniteAtomicSequence seq0 = other.atomicSequence("seq1", 1L, false);
 
             assertNotNull(seq0);
         }
 
-        assertNull(clientNode.atomicSequence("seq1", 1L, false));
-        assertNull(srvNode.atomicSequence("seq1", 1L, false));
+        assertNull(creator.atomicSequence("seq1", 1L, false));
+        assertNull(other.atomicSequence("seq1", 1L, false));
     }
 
     /**
@@ -105,12 +115,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testAtomicLong() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.atomicLong("long1", 1L, false));
+        testAtomicLong(clientNode, srvNode);
+        testAtomicLong(srvNode, clientNode);
+    }
 
-        try (IgniteAtomicLong cntr = clientNode.atomicLong("long1", 1L, true)) {
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testAtomicLong(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.atomicLong("long1", 1L, false));
+        assertNull(other.atomicLong("long1", 1L, false));
+
+        try (IgniteAtomicLong cntr = creator.atomicLong("long1", 1L, true)) {
             assertNotNull(cntr);
 
             assertEquals(1L, cntr.get());
@@ -119,7 +139,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertEquals(2L, cntr.get());
 
-            IgniteAtomicLong cntr0 = srvNode.atomicLong("long1", 1L, false);
+            IgniteAtomicLong cntr0 = other.atomicLong("long1", 1L, false);
 
             assertNotNull(cntr0);
 
@@ -130,8 +150,8 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
             assertEquals(3L, cntr.get());
         }
 
-        assertNull(clientNode.atomicLong("long1", 1L, false));
-        assertNull(srvNode.atomicLong("long1", 1L, false));
+        assertNull(creator.atomicLong("long1", 1L, false));
+        assertNull(other.atomicLong("long1", 1L, false));
     }
 
     /**
@@ -139,14 +159,24 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testSet() throws Exception {
         Ignite clientNode = clientIgnite();
-
         Ignite srvNode = serverNode();
 
-        assertNull(clientNode.set("set1", null));
+        testSet(clientNode, srvNode);
+        testSet(srvNode, clientNode);
+    }
+
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testSet(Ignite creator, Ignite other) throws Exception {
+        assertNull(creator.set("set1", null));
+        assertNull(other.set("set1", null));
 
         CollectionConfiguration colCfg = new CollectionConfiguration();
 
-        try (IgniteSet<Integer> set = clientNode.set("set1", colCfg)) {
+        try (IgniteSet<Integer> set = creator.set("set1", colCfg)) {
             assertNotNull(set);
 
             assertEquals(0, set.size());
@@ -157,7 +187,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertTrue(set.contains(1));
 
-            IgniteSet<Integer> set0 = srvNode.set("set1", null);
+            IgniteSet<Integer> set0 = other.set("set1", null);
 
             assertTrue(set0.contains(1));
 
@@ -167,6 +197,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             assertFalse(set.contains(1));
         }
+
+        assertNull(creator.set("set1", null));
+        assertNull(other.set("set1", null));
     }
 
     /**
@@ -174,12 +207,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testLatch() throws Exception {
         Ignite clientNode = clientIgnite();
+        Ignite srvNode = serverNode();
 
-        final Ignite srvNode = serverNode();
+        testLatch(clientNode, srvNode);
+        testLatch(srvNode, clientNode);
+    }
 
-        assertNull(clientNode.countDownLatch("latch1", 1, true, false));
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testLatch(Ignite creator, final Ignite other) throws Exception {
+        assertNull(creator.countDownLatch("latch1", 1, true, false));
+        assertNull(other.countDownLatch("latch1", 1, true, false));
 
-        try (IgniteCountDownLatch latch = clientNode.countDownLatch("latch1", 1, true, true)) {
+        try (IgniteCountDownLatch latch = creator.countDownLatch("latch1", 1, true, true)) {
             assertNotNull(latch);
 
             assertEquals(1, latch.count());
@@ -188,7 +231,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
                 @Override public Object call() throws Exception {
                     U.sleep(1000);
 
-                    IgniteCountDownLatch latch0 = srvNode.countDownLatch("latch1", 1, true, false);
+                    IgniteCountDownLatch latch0 = other.countDownLatch("latch1", 1, true, false);
 
                     assertEquals(1, latch0.count());
 
@@ -210,6 +253,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             fut.get();
         }
+
+        assertNull(creator.countDownLatch("latch1", 1, true, false));
+        assertNull(other.countDownLatch("latch1", 1, true, false));
     }
 
     /**
@@ -217,14 +263,22 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
      */
     public void testQueue() throws Exception {
         Ignite clientNode = clientIgnite();
+        Ignite srvNode = serverNode();
 
-        final Ignite srvNode = serverNode();
-
-        CollectionConfiguration colCfg = new CollectionConfiguration();
+        testQueue(clientNode, srvNode);
+        testQueue(srvNode, clientNode);
+    }
 
-        assertNull(clientNode.queue("q1", 0, null));
+    /**
+     * @param creator Creator node.
+     * @param other Other node.
+     * @throws Exception If failed.
+     */
+    private void testQueue(Ignite creator, final Ignite other) throws Exception {
+        assertNull(creator.queue("q1", 0, null));
+        assertNull(other.queue("q1", 0, null));
 
-        try (IgniteQueue<Integer> queue = clientNode.queue("q1", 0, colCfg)) {
+        try (IgniteQueue<Integer> queue = creator.queue("q1", 0, new CollectionConfiguration())) {
             assertNotNull(queue);
 
             queue.add(1);
@@ -235,7 +289,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
                 @Override public Object call() throws Exception {
                     U.sleep(1000);
 
-                    IgniteQueue<Integer> queue0 = srvNode.queue("q1", 0, null);
+                    IgniteQueue<Integer> queue0 = other.queue("q1", 0, null);
 
                     assertEquals(0, queue0.size());
 
@@ -255,6 +309,9 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
             fut.get();
         }
+
+        assertNull(creator.queue("q1", 0, null));
+        assertNull(other.queue("q1", 0, null));
     }
 
     /**


[26/28] incubator-ignite git commit: #ignite-964: change cache-api-example.js

Posted by iv...@apache.org.
#ignite-964: change cache-api-example.js


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

Branch: refs/heads/ignite-964-1
Commit: 242c21bb10ee0578055adeda1014a536e73ef88f
Parents: 7763a37
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jul 9 17:08:06 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jul 9 17:08:06 2015 +0300

----------------------------------------------------------------------
 examples/config/js/example-js-cache.xml         |   3 +
 .../examples/js/ExampleJsNodeStartup.java       |   4 +-
 examples/src/main/js/cache-api-example.js       | 103 ++++++++++++-------
 modules/nodejs/src/main/js/ignite.js            |   4 +-
 4 files changed, 74 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/242c21bb/examples/config/js/example-js-cache.xml
----------------------------------------------------------------------
diff --git a/examples/config/js/example-js-cache.xml b/examples/config/js/example-js-cache.xml
index 2599e38..e8ffc8a 100644
--- a/examples/config/js/example-js-cache.xml
+++ b/examples/config/js/example-js-cache.xml
@@ -31,6 +31,9 @@
     <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
         <property name="gridName" value="ServerNode" />
 
+        <!-- Set to true to enable distributed class loading for examples, default is false. -->
+        <property name="peerClassLoadingEnabled" value="true"/>
+
         <property name="connectorConfiguration">
             <bean class="org.apache.ignite.configuration.ConnectorConfiguration">
                 <property name="jettyPath" value="examples/config/js/rest-jetty.xml"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/242c21bb/examples/src/main/java/org/apache/ignite/examples/js/ExampleJsNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/js/ExampleJsNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/js/ExampleJsNodeStartup.java
index 6fa2e6c..0de6047 100644
--- a/examples/src/main/java/org/apache/ignite/examples/js/ExampleJsNodeStartup.java
+++ b/examples/src/main/java/org/apache/ignite/examples/js/ExampleJsNodeStartup.java
@@ -21,11 +21,11 @@ import org.apache.ignite.*;
 import org.apache.ignite.spi.discovery.tcp.internal.*;
 
 /**
- * Starts up an empty node with example compute configuration.
+ * Starts up an empty node with example node js configuration.
  */
 public class ExampleJsNodeStartup {
     /**
-     * Start up an empty node with example compute configuration.
+     * Start up an empty node with example node js configuration.
      *
      * @param args Command line arguments, none required.
      * @throws IgniteException If failed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/242c21bb/examples/src/main/js/cache-api-example.js
----------------------------------------------------------------------
diff --git a/examples/src/main/js/cache-api-example.js b/examples/src/main/js/cache-api-example.js
index 13368d5..d17276a 100644
--- a/examples/src/main/js/cache-api-example.js
+++ b/examples/src/main/js/cache-api-example.js
@@ -18,50 +18,81 @@
 var apacheIgnite = require("apache-ignite");
 var Ignition = apacheIgnite.Ignition;
 
-Ignition.start(['127.0.0.1:9095'], null, onConnect);
+/**
+  * This example demonstrates some of the cache rich API capabilities.
+  * <p>
+  * Remote nodes should always be started with special configuration file which
+  * enables P2P class loading: {@code 'ignite.{sh|bat} examples/config/js/example-js-cache.xml'}.
+  * <p>
+  * Alternatively you can run {@link ExampleJsNodeStartup} in another JVM which will
+  * start node with {@code examples/config/js/example-js-cache.xml} configuration.
+  */
+function main() {
+    /** Cache name. */
+    var cacheName = "ApiExampleCache";
+
+    /** Connect to node that started with {@code examples/config/js/example-js-cache.xml} configuration. */
+    Ignition.start(['127.0.0.1:9095'], null, onConnect);
+
+    function onConnect(err, ignite) {
+        console.log(">>> Cache API example started.");
+
+        // Create cache on server with cacheName.
+        ignite.getOrCreateCache(cacheName, function(err, cache) {
+                atomicMapOperations(ignite, cache);
+            });
+    }
 
-function onConnect(err, ignite) {
-    console.log(">>> Cache API example started.");
+    /**
+     * Demonstrates cache operations similar to {@link ConcurrentMap} API. Note that
+     * cache API is a lot richer than the JDK {@link ConcurrentMap}.
+     */
+    atomicMapOperations = function(ignite, cache) {
+        console.log(">>> Cache atomic map operation examples.");
 
-    ignite.getOrCreateCache("ApiExampleCache", function(err, cache) {
-            atomicMapOperations(cache);
+        cache.removeAllFromCache(function(err) {
+            // Put and return previous value.
+            cache.getAndPut(1, "1", onGetAndPut)
         });
-}
 
-/**
- * Demonstrates cache operations similar to {@link ConcurrentMap} API. Note that
- * cache API is a lot richer than the JDK {@link ConcurrentMap}.
- */
-atomicMapOperations = function(cache) {
-    console.log(">>> Cache atomic map operation examples.");
+        onGetAndPut = function(err, entry) {
+            console.log(">>> Get and put finished [result=" + entry + "]");
 
-    cache.removeAllFromCache(function(err) {
-        cache.getAndPut(1, "1", onGetAndPut.bind(null, cache))
-    });
-}
+            // Put and do not return previous value.
+            // Performs better when previous value is not needed.
+            cache.put(2, "2", onPut);
+        }
 
-function onGetAndPut(cache, err, entry) {
-    cache.put(2, "2", onPut.bind(null, cache));
-}
+        onPut = function(err) {
+            console.log(">>> Put finished.");
 
-function onPut(cache, err) {
-    cache.putIfAbsent(4, "44", onPutIfAbsent.bind(null, cache, true));
-}
+            // Put-if-absent.
+            cache.putIfAbsent(4, "44", onPutIfAbsent);
+        }
 
-function onPutIfAbsent(cache, expRes, err, res) {
-    if (expRes) {
-        cache.putIfAbsent(4, "44", onPutIfAbsent.bind(null, cache, false));
-    }
-    else {
-        cache.replaceValue(4, "55", "44", onReplaceValue.bind(null, cache, true));
+        onPutIfAbsent = function(err, res) {
+            console.log(">>> Put if absent finished [result=" + res + "]");
+
+            // Replace.
+            cache.replaceValue(4, "55", "44", onReplaceValue);
+        }
+
+        onReplaceValue = function(err, res) {
+            console.log(">>> Replace value finished [result=" + res + "]");
+
+            // Replace not correct value.
+            cache.replaceValue(4, "555", "44", onEnd);
+        }
+
+        onEnd = function(err) {
+            console.log(">>> Replace finished.");
+
+            // Destroying cache.
+            ignite.destroyCache(cacheName, function(err) {
+                    console.log(">>> End of Cache API example.");
+                });
+        }
     }
 }
 
-function onReplaceValue(cache, expRes, err, res) {
-    if (expRes) {
-        cache.replaceValue(4, "555", "44", onReplaceValue.bind(null, cache, false));
-    }
-    else {
-        console.log("End of the example.")
-    }
-}
\ No newline at end of file
+main();
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/242c21bb/modules/nodejs/src/main/js/ignite.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/ignite.js b/modules/nodejs/src/main/js/ignite.js
index 5dfb15b..a4a1dd9 100644
--- a/modules/nodejs/src/main/js/ignite.js
+++ b/modules/nodejs/src/main/js/ignite.js
@@ -59,7 +59,7 @@ Ignite.prototype.cache = function(cacheName) {
  * @param callback Callback with cache.
  */
 Ignite.prototype.getOrCreateCache = function(cacheName, callback) {
-    var onCreateCallback = function(err) {
+    var onCreateCallback = function(callback, err, res) {
         if (err !== null) {
             callback.call(null, err, null);
 
@@ -70,7 +70,7 @@ Ignite.prototype.getOrCreateCache = function(cacheName, callback) {
     }
 
     this._server.runCommand(new Command("getorcreatecache").addParam("cacheName", cacheName),
-        onCreateCallback.bind(this));
+        onCreateCallback.bind(this, callback));
 }
 
 /**


[14/28] incubator-ignite git commit: IGNITE-1026 - Dynamic cache start fixes

Posted by iv...@apache.org.
IGNITE-1026 - Dynamic cache start fixes


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

Branch: refs/heads/ignite-964-1
Commit: 8ff3619bae65018958c61a3299a415e595dd16c4
Parents: 378e344
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 1 18:51:51 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 1 18:51:51 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  2 +-
 .../IgniteCacheConfigurationTemplateTest.java   | 26 +++-----------------
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 12 ++-------
 .../IgniteDynamicClientCacheStartSelfTest.java  |  5 ++--
 4 files changed, 9 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 209946b..7103b1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -308,7 +308,7 @@ public interface Ignite extends AutoCloseable {
      * @param name Cache name.
      * @return Instance of the cache for the specified name.
      */
-    public <K, V> IgniteCache<K, V> cache(@Nullable String name);
+    @Nullable public <K, V> IgniteCache<K, V> cache(@Nullable String name);
 
     /**
      * Gets grid transactions facade.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
index 937a3b4..1085411 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
@@ -420,29 +420,9 @@ public class IgniteCacheConfigurationTemplateTest extends GridCommonAbstractTest
                 }
             }, IllegalStateException.class, null);
 
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE1);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
-
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE2);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
-
-            GridTestUtils.assertThrows(log, new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    ignite.cache(TEMPLATE3);
-
-                    return null;
-                }
-            }, IllegalArgumentException.class, null);
+            assertNull(ignite.cache(TEMPLATE1));
+            assertNull(ignite.cache(TEMPLATE2));
+            assertNull(ignite.cache(TEMPLATE3));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index e7b0ba5..cd19703 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -310,11 +310,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                 f.get();
 
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    return kernal0.cache(DYNAMIC_CACHE_NAME);
-                }
-            }, IllegalArgumentException.class, null);
+            assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
 
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -368,11 +364,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
 
-                GridTestUtils.assertThrows(log, new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        return kernal0.cache(DYNAMIC_CACHE_NAME);
-                    }
-                }, IllegalArgumentException.class, null);
+                assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8ff3619b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
index 24935c7..abada13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import javax.cache.*;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
@@ -139,7 +140,7 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
 
                 return null;
             }
-        }, IgniteException.class, null);
+        }, CacheException.class, null);
 
         checkCache(ignite1, cacheName, false, false);
 
@@ -149,7 +150,7 @@ public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTes
 
                 return null;
             }
-        }, IgniteException.class, null);
+        }, CacheException.class, null);
 
         checkCache(ignite1, cacheName, false, false);
     }


[02/28] incubator-ignite git commit: Merge branches 'ignite-1026' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026

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


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

Branch: refs/heads/ignite-964-1
Commit: d6dd707ba0806d3a01dc7acc09d6948226c0177d
Parents: eef2b37 ad0a026
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 12:37:21 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 12:37:21 2015 -0700

----------------------------------------------------------------------
 .../s3/S3CheckpointManagerSelfTest.java         |   2 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   4 +-
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |   2 +-
 .../s3/S3SessionCheckpointSelfTest.java         |   2 +-
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |   2 +-
 .../affinity/AffinityTopologyVersion.java       |   7 -
 .../processors/cache/GridCacheUtils.java        |   9 +
 .../processors/cache/IgniteCacheProxy.java      |   5 +
 .../distributed/dht/GridDhtLocalPartition.java  |  56 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../dht/GridDhtPartitionsReservation.java       | 292 +++++++++
 .../cache/distributed/dht/GridReservable.java   |  35 +
 .../dht/preloader/GridDhtPartitionMap.java      |  26 +-
 .../cache/query/GridCacheQueryManager.java      |  33 -
 .../cache/query/GridCacheTwoStepQuery.java      |  22 +-
 .../processors/query/GridQueryIndexing.java     |  14 +-
 .../processors/query/GridQueryProcessor.java    |  21 +-
 .../messages/GridQueryNextPageResponse.java     |  34 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 111 +++-
 .../apache/ignite/internal/util/GridDebug.java  |  19 +
 .../communication/tcp/TcpCommunicationSpi.java  |  42 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  35 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  42 +-
 .../tcp/internal/TcpDiscoveryNode.java          |  18 +
 .../apache/ignite/internal/GridSelfTest.java    |  20 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  24 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |  30 +-
 .../IgniteCacheAtomicStopBusySelfTest.java      |   8 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  19 +-
 ...gniteCacheTransactionalStopBusySelfTest.java |   8 +-
 .../DataStreamerMultiThreadedSelfTest.java      |   3 +
 .../junits/GridTestKernalContext.java           |   2 +-
 .../junits/common/GridCommonAbstractTest.java   |   8 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  79 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  49 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 332 +++++++---
 .../query/h2/twostep/GridMergeIndex.java        |  17 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   7 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 650 ++++++++++++++++---
 .../query/h2/twostep/GridResultPage.java        |  21 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++++
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  36 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 383 +++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 46 files changed, 2577 insertions(+), 381 deletions(-)
----------------------------------------------------------------------



[09/28] incubator-ignite git commit: # IGNITE-1026 - Dynamic cache start fix

Posted by iv...@apache.org.
# IGNITE-1026 - Dynamic cache start 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/9adc180b
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9adc180b
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9adc180b

Branch: refs/heads/ignite-964-1
Commit: 9adc180bcf1320bfb92d387e64c82a2f5b1029b6
Parents: 08ea4cc
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Fri Jun 19 15:50:30 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Fri Jun 19 15:50:30 2015 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java    | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9adc180b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 3ee260d..6c7c511 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -77,7 +77,6 @@ import javax.management.*;
 import java.io.*;
 import java.lang.management.*;
 import java.lang.reflect.*;
-import java.security.*;
 import java.text.*;
 import java.util.*;
 import java.util.concurrent.*;
@@ -2248,7 +2247,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         guard();
 
         try {
-            return ctx.cache().publicJCache(name, true);
+            return ctx.cache().publicJCache(name, false);
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);


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

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


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

Branch: refs/heads/ignite-964-1
Commit: 3089ace6c9de37765b6f4fa0697b1430f4cb417e
Parents: d27453f f72b291
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Sun Jul 5 11:55:10 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Sun Jul 5 11:55:10 2015 -0700

----------------------------------------------------------------------
 bin/ignite.bat                                  |   8 +-
 bin/ignite.sh                                   |   6 +-
 bin/include/parseargs.bat                       |   1 +
 bin/include/parseargs.sh                        |   3 +
 .../startup/cmdline/CommandLineStartup.java     |   3 +-
 .../startup/cmdline/CommandLineTransformer.java |   9 +
 .../GridCachePartitionedFailoverSelfTest.java   |   5 +
 modules/kafka/pom.xml                           | 116 ++++++
 .../ignite/stream/kafka/KafkaStreamer.java      | 220 +++++++++++
 .../kafka/IgniteKafkaStreamerSelfTestSuite.java |  37 ++
 .../stream/kafka/KafkaEmbeddedBroker.java       | 378 +++++++++++++++++++
 .../kafka/KafkaIgniteStreamerSelfTest.java      | 227 +++++++++++
 .../ignite/stream/kafka/SimplePartitioner.java  |  53 +++
 .../util/spring/IgniteSpringHelperImpl.java     |  72 +++-
 .../IgniteExcludeInConfigurationTest.java       |  78 ++++
 .../org/apache/ignite/spring/sprint-exclude.xml |  57 +++
 .../testsuites/IgniteSpringTestSuite.java       |   2 +
 .../ignite/visor/commands/VisorConsole.scala    |   3 +-
 .../visor/commands/open/VisorOpenCommand.scala  | 319 ++++++++++++++++
 .../scala/org/apache/ignite/visor/visor.scala   | 230 +----------
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |   2 +
 .../commands/kill/VisorKillCommandSpec.scala    |   1 +
 .../commands/start/VisorStartCommandSpec.scala  |   1 +
 .../commands/tasks/VisorTasksCommandSpec.scala  |   1 +
 .../commands/vvm/VisorVvmCommandSpec.scala      |   1 +
 pom.xml                                         |   1 +
 26 files changed, 1593 insertions(+), 241 deletions(-)
----------------------------------------------------------------------



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

Posted by iv...@apache.org.
Merge branch 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-1026


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

Branch: refs/heads/ignite-964-1
Commit: d38ad8ce2fe85ac61bd050fa696348f1e6cc29fa
Parents: 6f50ad9 ea90d86
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 16:05:12 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 16:05:12 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteCache.java     |   5 +
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../managers/communication/GridIoManager.java   | 124 +++++++--
 .../managers/communication/GridIoMessage.java   |  15 +-
 .../managers/communication/GridIoPolicy.java    |  32 +--
 .../eventstorage/GridEventStorageManager.java   |   2 +-
 .../processors/cache/CacheOperationContext.java |  44 +++-
 .../internal/processors/cache/CacheType.java    |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  91 ++++---
 .../processors/cache/GridCacheAtomicFuture.java |  12 +-
 .../processors/cache/GridCacheContext.java      |   4 +-
 .../processors/cache/GridCacheIoManager.java    |  12 +-
 .../processors/cache/GridCacheMvccManager.java  |   8 +-
 .../processors/cache/GridCacheProxyImpl.java    |  10 +-
 .../processors/cache/GridCacheSwapManager.java  | 257 ++++++++++++-------
 .../processors/cache/GridCacheUtils.java        |  42 +++
 .../processors/cache/IgniteCacheProxy.java      |  36 ++-
 .../GridDistributedTxFinishRequest.java         |  11 +-
 .../GridDistributedTxPrepareRequest.java        |   9 +-
 .../GridDistributedTxRemoteAdapter.java         |   3 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |   3 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   3 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   3 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |   5 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |  15 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 177 +++++++++++--
 .../near/GridNearTxFinishRequest.java           |   3 +-
 .../cache/distributed/near/GridNearTxLocal.java |   3 +-
 .../distributed/near/GridNearTxRemote.java      |   5 +-
 .../cache/transactions/IgniteInternalTx.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |  11 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../datastructures/GridCacheAtomicLongImpl.java |  25 +-
 .../GridCacheAtomicSequenceImpl.java            |  11 +-
 .../GridCacheAtomicStampedImpl.java             |  21 +-
 .../GridCacheCountDownLatchImpl.java            |  16 +-
 .../internal/processors/igfs/IgfsContext.java   |   5 +-
 .../plugin/IgnitePluginProcessor.java           |   3 +-
 .../plugin/extensions/communication/IoPool.java |  42 +++
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   5 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   5 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   2 +-
 .../communication/GridIoManagerSelfTest.java    |   2 +-
 .../cache/IgniteInternalCacheTypesTest.java     |   3 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    | 147 +++++++++++
 .../dht/IgniteCachePutRetryAtomicSelfTest.java  |  34 +++
 ...gniteCachePutRetryTransactionalSelfTest.java |  74 ++++++
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   5 +-
 .../GridCacheEvictionFilterSelfTest.java        |   2 -
 .../inmemory/GridTestSwapSpaceSpi.java          |   3 +-
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   8 +-
 .../processors/query/h2/opt/GridH2Table.java    |   2 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   4 +-
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 -
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |   5 +
 modules/urideploy/pom.xml                       |  14 -
 60 files changed, 1053 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d38ad8ce/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------


[19/28] incubator-ignite git commit: moved link to the project site to the top of the log

Posted by iv...@apache.org.
moved link to the project site to the top of the log


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

Branch: refs/heads/ignite-964-1
Commit: 459d7022f8685858eb01792f2c99db5f5efc8e9a
Parents: 82f4992
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:39:09 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:39:09 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/IgniteKernal.java     | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/459d7022/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 30931fa..c12d2cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1486,10 +1486,12 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     "  /  _/ ___/ |/ /  _/_  __/ __/ ",
                     " _/ // (7 7    // /  / / / _/   ",
                     "/___/\\___/_/|_/___/ /_/ /___/  ",
-                    " ",
+                    "",
                     ver,
                     COPYRIGHT,
                     "",
+                    "Ignite documentation: " + "http://" + SITE,
+                    "",
                     "Quiet mode.");
 
                 if (fileName != null)
@@ -1508,7 +1510,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                         ">>> /___/\\___/_/|_/___/ /_/ /___/   " + NL +
                         ">>> " + NL +
                         ">>> " + ver + NL +
-                        ">>> " + COPYRIGHT + NL
+                        ">>> " + COPYRIGHT + NL +
+                        ">>> " + NL +
+                        ">>> Ignite documentation: " + "http://" + SITE + NL
                 );
             }
         }
@@ -1557,8 +1561,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     ">>> Local node addresses: " + U.addressesAsString(locNode) + NL +
                     ">>> Local ports: " + sb + NL;
 
-            str += ">>> Ignite documentation: http://" + SITE + NL;
-
             log.info(str);
         }
     }


[17/28] incubator-ignite git commit: IGNITE-1026 - Count down latch fix

Posted by iv...@apache.org.
IGNITE-1026 - Count down latch 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/d27453fd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d27453fd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d27453fd

Branch: refs/heads/ignite-964-1
Commit: d27453fdcdea4612a7ba4d0ef6b0168eb5569ed0
Parents: d38ad8c
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jul 2 17:52:56 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jul 2 17:52:56 2015 -0700

----------------------------------------------------------------------
 .../IgniteCountDownLatchAbstractSelfTest.java           | 12 +++++++++---
 1 file changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d27453fd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
index 80e6123..01ffaf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
@@ -19,9 +19,9 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
 
@@ -133,7 +133,13 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
      *
      * @throws Exception If failed.
      */
-    protected void checkRemovedLatch(IgniteCountDownLatch latch) throws Exception {
+    protected void checkRemovedLatch(final IgniteCountDownLatch latch) throws Exception {
+        assert GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                return latch.removed();
+            }
+        }, 5000);
+
         assert latch.removed();
 
         assert latch.count() == 0;
@@ -177,7 +183,7 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
      */
     private void checkAwait() throws Exception {
         // Check only 'false' cases here. Successful await is tested over the grid.
-        IgniteCountDownLatch latch = createLatch("await", 5, true);
+        IgniteCountDownLatch latch = createLatch("await", 5, false);
 
         assert !latch.await(10);
         assert !latch.await(10, MILLISECONDS);


[21/28] incubator-ignite git commit: release notes

Posted by iv...@apache.org.
release notes


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

Branch: refs/heads/ignite-964-1
Commit: 064d079ce4f7cc501936b95a1eaae144c6b823bb
Parents: fa5cb91
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jul 8 19:51:33 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jul 8 19:51:33 2015 +0300

----------------------------------------------------------------------
 RELEASE_NOTES.txt | 10 ++++++++++
 1 file changed, 10 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/064d079c/RELEASE_NOTES.txt
----------------------------------------------------------------------
diff --git a/RELEASE_NOTES.txt b/RELEASE_NOTES.txt
index 3a5043e..bcfed27 100644
--- a/RELEASE_NOTES.txt
+++ b/RELEASE_NOTES.txt
@@ -1,6 +1,16 @@
 Apache Ignite Release Notes
 ===========================
 
+Apache Ignite In-Memory Data Fabric 1.3
+---------------------------------------
+
+* Added auto-retries for cache operations in recoverable cases.
+* Fixed several issues with JTA integration.
+* Stability fixes for TCP discovery SPI.
+* Stability fixes for onheap and offheap SQL queries.
+* Bug fixes in In-Memory Accelerator For Apache Hadoop.
+* Many stability and fault-tolerance fixes.
+
 Apache Ignite In-Memory Data Fabric 1.2
 ---------------------------------------
 


[04/28] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1026' into ignite-1026

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


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

Branch: refs/heads/ignite-964-1
Commit: 2f089f1bda24b0e54735e5f8dccfc372647e1c54
Parents: d6dd707 bb73b66
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Thu Jun 18 16:03:40 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Thu Jun 18 16:03:40 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++++-----
 .../processors/cache/GridCacheProcessor.java    | 31 ++++++++++++--------
 .../datastructures/DataStructuresProcessor.java |  2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  4 +--
 4 files changed, 30 insertions(+), 23 deletions(-)
----------------------------------------------------------------------