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

[1/2] incubator-ignite git commit: ignite-1085: fixed

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1085 4031db76d -> 7510aff0e


ignite-1085: fixed


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

Branch: refs/heads/ignite-1085
Commit: 121ddafe9382cb875f2560263778ccbb6cb711da
Parents: 4031db7
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 9 12:34:15 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 9 12:34:15 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  59 ++++-----
 .../processors/cache/GridCacheProcessor.java    | 122 ++++++++-----------
 .../ignite/internal/util/IgniteUtils.java       |  31 ++++-
 3 files changed, 103 insertions(+), 109 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/121ddafe/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a8ce8ff..77dfa4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -64,9 +64,6 @@ import static org.apache.ignite.plugin.segmentation.SegmentationPolicy.*;
  * Discovery SPI manager.
  */
 public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
-    /** Fake key for {@code null}-named caches. Used inside {@link DiscoCache}. */
-    private static final String NULL_CACHE_NAME = UUID.randomUUID().toString();
-
     /** Metrics update frequency. */
     private static final long METRICS_UPDATE_FREQ = 3000;
 
@@ -179,8 +176,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     private ConcurrentMap<Class<?>, List<CustomEventListener<DiscoveryCustomMessage>>> customEvtLsnrs =
         new ConcurrentHashMap8<>();
 
-    /** Map of dynamic cache filters. */
-    private Map<String, CachePredicate> registeredCaches = new HashMap<>();
+    /**
+     * Map of dynamic cache filters. As long as this collection doesn't accept {@code null}s use {@link
+     * IgniteUtils#maskNull(String)} before passing raw cache names to it.
+     */
+    private final ConcurrentHashMap<String, CachePredicate> registeredCaches = new ConcurrentHashMap<>();
 
     /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
@@ -235,8 +235,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         boolean nearEnabled,
         boolean loc
     ) {
-        if (!registeredCaches.containsKey(cacheName))
-            registeredCaches.put(cacheName, new CachePredicate(filter, nearEnabled, loc));
+        registeredCaches.putIfAbsent(U.maskNull(cacheName), new CachePredicate(filter, nearEnabled, loc));
     }
 
     /**
@@ -245,7 +244,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param cacheName Cache name.
      */
     public void removeCacheFilter(String cacheName) {
-        registeredCaches.remove(cacheName);
+        registeredCaches.remove(U.maskNull(cacheName));
     }
 
     /**
@@ -256,7 +255,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param nearEnabled Near enabled flag.
      */
     public void addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(U.maskNull(cacheName));
 
         if (pred != null)
             pred.addClientNode(clientNodeId, nearEnabled);
@@ -275,7 +274,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 if (res == null)
                     res = U.newHashMap(registeredCaches.size());
 
-                res.put(entry.getKey(), new HashMap<>(pred.clientNodes));
+                res.put(U.unmaskNull(entry.getKey()), new HashMap<>(pred.clientNodes));
             }
         }
 
@@ -1360,7 +1359,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node is a cache data node.
      */
     public boolean cacheAffinityNode(ClusterNode node, String cacheName) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(U.maskNull(cacheName));
 
         return pred != null && pred.dataNode(node);
     }
@@ -1371,7 +1370,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has near cache enabled.
      */
     public boolean cacheNearNode(ClusterNode node, String cacheName) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(U.maskNull(cacheName));
 
         return pred != null && pred.nearNode(node);
     }
@@ -1382,7 +1381,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has client cache (without near cache).
      */
     public boolean cacheClientNode(ClusterNode node, String cacheName) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(U.maskNull(cacheName));
 
         return pred != null && pred.clientNode(node);
     }
@@ -1393,7 +1392,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return If cache with the given name is accessible on the given node.
      */
     public boolean cacheNode(ClusterNode node, String cacheName) {
-        CachePredicate pred = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(U.maskNull(cacheName));
 
         return pred != null && pred.cacheNode(node);
     }
@@ -2166,13 +2165,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         /**
          * Cached alive nodes list. As long as this collection doesn't accept {@code null}s use {@link
-         * #maskNull(String)} before passing raw cache names to it.
+         * IgniteUtils#maskNull(String)} before passing raw cache names to it.
          */
         private final ConcurrentMap<String, Collection<ClusterNode>> aliveCacheNodes;
 
         /**
          * Cached alive remote nodes list. As long as this collection doesn't accept {@code null}s use {@link
-         * #maskNull(String)} before passing raw cache names to it.
+         * IgniteUtils#maskNull(String)} before passing raw cache names to it.
          */
         private final ConcurrentMap<String, Collection<ClusterNode>> aliveRmtCacheNodes;
 
@@ -2240,7 +2239,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 boolean hasCaches = false;
 
                 for (Map.Entry<String, CachePredicate> entry : registeredCaches.entrySet()) {
-                    String cacheName = entry.getKey();
+                    String maskedCacheName = entry.getKey();
+                    String cacheName = U.unmaskNull(maskedCacheName);
 
                     CachePredicate filter = entry.getValue();
 
@@ -2253,7 +2253,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         addToMap(cacheMap, cacheName, node);
 
                         if (alive(node.id()))
-                            addToMap(aliveCacheNodes, maskNull(cacheName), node);
+                            addToMap(aliveCacheNodes, maskedCacheName, node);
 
                         if (filter.dataNode(node))
                             addToMap(dhtNodesMap, cacheName, node);
@@ -2265,7 +2265,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                             addToMap(rmtCacheMap, cacheName, node);
 
                             if (alive(node.id()))
-                                addToMap(aliveRmtCacheNodes, maskNull(cacheName), node);
+                                addToMap(aliveRmtCacheNodes, maskedCacheName, node);
                         }
 
                         hasCaches = true;
@@ -2449,7 +2449,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return Collection of nodes.
          */
         Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, aliveCacheNodes.get(maskNull(cacheName)));
+            return filter(topVer, aliveCacheNodes.get(U.maskNull(cacheName)));
         }
 
         /**
@@ -2460,7 +2460,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return Collection of nodes.
          */
         Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, aliveRmtCacheNodes.get(maskNull(cacheName)));
+            return filter(topVer, aliveRmtCacheNodes.get(U.maskNull(cacheName)));
         }
 
         /**
@@ -2528,11 +2528,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @param exclNode Node to exclude.
          */
         private void filterNodeMap(ConcurrentMap<String, Collection<ClusterNode>> map, final ClusterNode exclNode) {
-            for (String cacheName : registeredCaches.keySet()) {
-                String maskedName = maskNull(cacheName);
+            for (String maskedCacheName : registeredCaches.keySet()) {
 
                 while (true) {
-                    Collection<ClusterNode> oldNodes = map.get(maskedName);
+                    Collection<ClusterNode> oldNodes = map.get(maskedCacheName);
 
                     if (oldNodes == null || oldNodes.isEmpty())
                         break;
@@ -2542,23 +2541,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     if (!newNodes.remove(exclNode))
                         break;
 
-                    if (map.replace(maskedName, oldNodes, newNodes))
+                    if (map.replace(maskedCacheName, oldNodes, newNodes))
                         break;
                 }
             }
         }
 
         /**
-         * Replaces {@code null} with {@code NULL_CACHE_NAME}.
-         *
-         * @param cacheName Cache name.
-         * @return Masked name.
-         */
-        private String maskNull(@Nullable String cacheName) {
-            return cacheName == null ? NULL_CACHE_NAME : cacheName;
-        }
-
-        /**
          * @param topVer Topology version.
          * @param nodes Nodes.
          * @return Filtered collection (potentially empty, but never {@code null}).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/121ddafe/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 8124594..0f883c1 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
@@ -82,9 +82,6 @@ import static org.apache.ignite.transactions.TransactionIsolation.*;
  */
 @SuppressWarnings("unchecked")
 public class GridCacheProcessor extends GridProcessorAdapter {
-    /** Null cache name. */
-    private static final String NULL_NAME = U.id8(UUID.randomUUID());
-
     /** Shared cache context. */
     private GridCacheSharedContext<?, ?> sharedCtx;
 
@@ -561,8 +558,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         if (igfsCfgs != null) {
             for (FileSystemConfiguration igfsCfg : igfsCfgs) {
-                internalCaches.add(maskNull(igfsCfg.getMetaCacheName()));
-                internalCaches.add(maskNull(igfsCfg.getDataCacheName()));
+                internalCaches.add(U.maskNull(igfsCfg.getMetaCacheName()));
+                internalCaches.add(U.maskNull(igfsCfg.getDataCacheName()));
             }
         }
 
@@ -591,7 +588,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             cfgs[i] = cfg; // Replace original configuration value.
 
-            String masked = maskNull(cfg.getName());
+            String masked = U.maskNull(cfg.getName());
 
             if (registeredCaches.containsKey(masked)) {
                 String cacheName = cfg.getName();
@@ -610,7 +607,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cacheType = CacheType.UTILITY;
             else if (CU.isMarshallerCache(cfg.getName()))
                 cacheType = CacheType.MARSHALLER;
-            else if (internalCaches.contains(maskNull(cfg.getName())))
+            else if (internalCaches.contains(U.maskNull(cfg.getName())))
                 cacheType = CacheType.INTERNAL;
             else
                 cacheType = CacheType.USER;
@@ -690,7 +687,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                         String name = ccfg.getName();
 
-                        caches.put(maskNull(name), cache);
+                        caches.put(U.maskNull(name), cache);
 
                         startCache(cache);
 
@@ -775,11 +772,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     String name = ccfg.getName();
 
-                    caches.put(maskNull(name), cache);
+                    caches.put(U.maskNull(name), cache);
 
                     startCache(cache);
 
-                    jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
+                    jCacheProxies.put(U.maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
                 }
             }
         }
@@ -829,7 +826,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Wait for caches in SYNC preload mode.
         for (CacheConfiguration cfg : ctx.config().getCacheConfiguration()) {
-            GridCacheAdapter cache = caches.get(maskNull(cfg.getName()));
+            GridCacheAdapter cache = caches.get(U.maskNull(cfg.getName()));
 
             if (cache != null) {
                 if (cfg.getRebalanceMode() == SYNC) {
@@ -852,7 +849,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.remove(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = caches.remove(U.maskNull(cacheName));
 
             if (cache != null)
                 stopCache(cache, cancel);
@@ -891,7 +888,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         cacheStartedLatch.countDown();
 
         for (String cacheName : stopSeq) {
-            GridCacheAdapter<?, ?> cache = caches.get(maskNull(cacheName));
+            GridCacheAdapter<?, ?> cache = caches.get(U.maskNull(cacheName));
 
             if (cache != null)
                 onKernalStop(cache, cancel);
@@ -1002,7 +999,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         ctx.kernalContext().continuous().onCacheStop(ctx);
 
         U.stopLifecycleAware(log, lifecycleAwares(cache.configuration(), ctx.jta().tmLookup(),
-            ctx.store().configuredStore()));
+                                                  ctx.store().configuredStore()));
 
         if (log.isInfoEnabled())
             log.info("Stopped cache: " + cache.name());
@@ -1376,7 +1373,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         return F.viewReadOnly(registeredCaches.keySet(),
             new IgniteClosure<String, String>() {
                 @Override public String apply(String s) {
-                    return unmaskNull(s);
+                    return U.unmaskNull(s);
                 }
             });
     }
@@ -1388,7 +1385,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache mode.
      */
     public CacheMode cacheMode(String cacheName) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(cacheName));
 
         return desc != null ? desc.cacheConfiguration().getCacheMode() : null;
     }
@@ -1399,7 +1396,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("IfMayBeConditional")
     public boolean dynamicCacheRegistered(DynamicCacheChangeRequest req) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(req.cacheName()));
 
         if (desc != null) {
             if (desc.deploymentId().equals(req.deploymentId())) {
@@ -1510,7 +1507,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             sharedCtx.addCacheContext(cacheCtx);
 
-            caches.put(maskNull(cacheCtx.name()), cacheCtx.cache());
+            caches.put(U.maskNull(cacheCtx.name()), cacheCtx.cache());
 
             startCache(cacheCtx.cache());
             onKernalStart(cacheCtx.cache());
@@ -1524,7 +1521,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         assert req.stop();
 
         // Break the proxy before exchange future is done.
-        IgniteCacheProxy<?, ?> proxy = jCacheProxies.get(maskNull(req.cacheName()));
+        IgniteCacheProxy<?, ?> proxy = jCacheProxies.get(U.maskNull(req.cacheName()));
 
         if (proxy != null)
             proxy.gate().block();
@@ -1537,7 +1534,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         assert req.stop();
 
         // Break the proxy before exchange future is done.
-        IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(maskNull(req.cacheName()));
+        IgniteCacheProxy<?, ?> proxy = jCacheProxies.remove(U.maskNull(req.cacheName()));
 
         if (proxy != null)
             proxy.gate().onStopped();
@@ -1549,7 +1546,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     public void prepareCacheStop(DynamicCacheChangeRequest req) {
         assert req.stop();
 
-        GridCacheAdapter<?, ?> cache = caches.remove(maskNull(req.cacheName()));
+        GridCacheAdapter<?, ?> cache = caches.remove(U.maskNull(req.cacheName()));
 
         if (cache != null) {
             GridCacheContext<?, ?> ctx = cache.context();
@@ -1583,7 +1580,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             if (F.eq(cacheCtx.startTopologyVersion(), topVer)) {
                 cacheCtx.preloader().onInitialExchangeComplete(err);
 
-                String masked = maskNull(cacheCtx.name());
+                String masked = U.maskNull(cacheCtx.name());
 
                 jCacheProxies.put(masked, new IgniteCacheProxy(cache.context(), cache, null, false));
             }
@@ -1591,7 +1588,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         if (!F.isEmpty(reqs) && err == null) {
             for (DynamicCacheChangeRequest req : reqs) {
-                String masked = maskNull(req.cacheName());
+                String masked = U.maskNull(req.cacheName());
 
                 if (req.stop()) {
                     stopGateway(req);
@@ -1613,7 +1610,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param req Request to complete future for.
      */
     public void completeStartFuture(DynamicCacheChangeRequest req) {
-        DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
+        DynamicCacheStartFuture fut = (DynamicCacheStartFuture)pendingFuts.get(U.maskNull(req.cacheName()));
 
         assert req.deploymentId() != null;
         assert fut == null || fut.deploymentId != null;
@@ -1707,7 +1704,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     assert ccfg != null : req;
 
-                    DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
+                    DynamicCacheDescriptor existing = registeredTemplates.get(U.maskNull(req.cacheName()));
 
                     if (existing == null) {
                         DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
@@ -1717,13 +1714,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                             true,
                             req.deploymentId());
 
-                        registeredTemplates.put(maskNull(req.cacheName()), desc);
+                        registeredTemplates.put(U.maskNull(req.cacheName()), desc);
                     }
 
                     continue;
                 }
 
-                DynamicCacheDescriptor existing = registeredCaches.get(maskNull(req.cacheName()));
+                DynamicCacheDescriptor existing = registeredCaches.get(U.maskNull(req.cacheName()));
 
                 if (req.start() && !req.clientStartOnly()) {
                     CacheConfiguration ccfg = req.startCacheConfiguration();
@@ -1755,7 +1752,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         if (req.initiatingNodeId() == null)
                             desc.staticallyConfigured(true);
 
-                        registeredCaches.put(maskNull(req.cacheName()), desc);
+                        registeredCaches.put(U.maskNull(req.cacheName()), desc);
 
                         ctx.discovery().setCacheFilter(
                             req.cacheName(),
@@ -1921,7 +1918,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         assert ccfg != null || nearCfg != null;
 
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(cacheName));
 
         DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
 
@@ -2039,7 +2036,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             try {
                 if (req.stop()) {
-                    DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+                    DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(req.cacheName()));
 
                     if (desc == null)
                         // No-op.
@@ -2059,7 +2056,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     continue;
 
                 DynamicCacheStartFuture old = (DynamicCacheStartFuture)pendingFuts.putIfAbsent(
-                    maskNull(req.cacheName()), fut);
+                    U.maskNull(req.cacheName()), fut);
 
                 if (old != null) {
                     if (req.start()) {
@@ -2118,20 +2115,20 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 assert ccfg != null : req;
 
-                DynamicCacheDescriptor desc = registeredTemplates.get(maskNull(req.cacheName()));
+                DynamicCacheDescriptor desc = registeredTemplates.get(U.maskNull(req.cacheName()));
 
                 if (desc == null) {
                     DynamicCacheDescriptor templateDesc =
                         new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), true, req.deploymentId());
 
-                    DynamicCacheDescriptor old = registeredTemplates.put(maskNull(ccfg.getName()), templateDesc);
+                    DynamicCacheDescriptor old = registeredTemplates.put(U.maskNull(ccfg.getName()), templateDesc);
 
                     assert old == null :
                         "Dynamic cache map was concurrently modified [new=" + templateDesc + ", old=" + old + ']';
                 }
 
                 TemplateConfigurationFuture fut =
-                    (TemplateConfigurationFuture)pendingTemplateFuts.get(maskNull(ccfg.getName()));
+                    (TemplateConfigurationFuture)pendingTemplateFuts.get(U.maskNull(ccfg.getName()));
 
                 if (fut != null && fut.deploymentId().equals(req.deploymentId()))
                     fut.onDone();
@@ -2139,13 +2136,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 continue;
             }
 
-            DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+            DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(req.cacheName()));
 
             if (req.start()) {
                 CacheConfiguration ccfg = req.startCacheConfiguration();
 
                 DynamicCacheStartFuture startFut = (DynamicCacheStartFuture)pendingFuts.get(
-                    maskNull(ccfg.getName()));
+                    U.maskNull(ccfg.getName()));
 
                 // Check if cache with the same name was concurrently started form different node.
                 if (desc != null) {
@@ -2178,7 +2175,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     DynamicCacheDescriptor startDesc =
                         new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId());
 
-                    DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc);
+                    DynamicCacheDescriptor old = registeredCaches.put(U.maskNull(ccfg.getName()), startDesc);
 
                     assert old == null :
                         "Dynamic cache map was concurrently modified [new=" + startDesc + ", old=" + old + ']';
@@ -2197,7 +2194,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             else {
                 if (desc == null) {
                     // If local node initiated start, fail the start future.
-                    DynamicCacheStartFuture changeFut = (DynamicCacheStartFuture)pendingFuts.get(maskNull(req.cacheName()));
+                    DynamicCacheStartFuture changeFut = (DynamicCacheStartFuture)pendingFuts.get(
+                        U.maskNull(req.cacheName()));
 
                     if (changeFut != null && changeFut.deploymentId().equals(req.deploymentId())) {
                         // No-op.
@@ -2464,7 +2462,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
          * it is called from synchronization block within Swap SPI.
          */
 
-        GridCacheAdapter cache = caches.get(maskNull(CU.cacheNameForSwapSpaceName(spaceName)));
+        GridCacheAdapter cache = caches.get(U.maskNull(CU.cacheNameForSwapSpaceName(spaceName)));
 
         assert cache != null : "Failed to resolve cache name for swap space name: " + spaceName;
 
@@ -2548,7 +2546,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting cache for name: " + name);
 
-        IgniteCacheProxy<K, V> jcache = (IgniteCacheProxy<K, V>)jCacheProxies.get(maskNull(name));
+        IgniteCacheProxy<K, V> jcache = (IgniteCacheProxy<K, V>)jCacheProxies.get(U.maskNull(name));
 
         return jcache == null ? null : jcache.internalProxy();
     }
@@ -2563,7 +2561,7 @@ 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));
+        IgniteCache<K, V> jcache = (IgniteCache<K, V>)jCacheProxies.get(U.maskNull(name));
 
         if (jcache == null)
             jcache = startJCache(name, true);
@@ -2626,7 +2624,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting public cache for name: " + name);
 
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(name));
 
         if (desc == null || desc.cancelled())
             throw new IllegalArgumentException("Cache is not started: " + name);
@@ -2634,7 +2632,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (!desc.cacheType().userCache())
             throw new IllegalStateException("Failed to get cache because it is a system cache: " + name);
 
-        IgniteCacheProxy<K, V> jcache = (IgniteCacheProxy<K, V>)jCacheProxies.get(maskNull(name));
+        IgniteCacheProxy<K, V> jcache = (IgniteCacheProxy<K, V>)jCacheProxies.get(U.maskNull(name));
 
         if (jcache == null)
             throw new IllegalArgumentException("Cache is not started: " + name);
@@ -2669,7 +2667,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting public cache for name: " + cacheName);
 
-        String masked = maskNull(cacheName);
+        String masked = U.maskNull(cacheName);
 
         IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
@@ -2694,7 +2692,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private IgniteCacheProxy startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
         checkEmptyTransactions();
 
-        String masked = maskNull(cacheName);
+        String masked = U.maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
 
@@ -2738,7 +2736,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache configuration.
      */
     public CacheConfiguration cacheConfiguration(String name) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(name));
 
         if (desc == null || desc.cancelled())
             throw new IllegalStateException("Cache doesn't exist: " + name);
@@ -2751,7 +2749,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     public void addCacheConfiguration(CacheConfiguration cacheCfg) throws IgniteCheckedException {
-        String masked = maskNull(cacheCfg.getName());
+        String masked = U.maskNull(cacheCfg.getName());
 
         DynamicCacheDescriptor desc = registeredTemplates.get(masked);
 
@@ -2771,7 +2769,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         TemplateConfigurationFuture fut = new TemplateConfigurationFuture(req.cacheName(), req.deploymentId());
 
         TemplateConfigurationFuture old =
-            (TemplateConfigurationFuture)pendingTemplateFuts.putIfAbsent(maskNull(cacheCfg.getName()), fut);
+            (TemplateConfigurationFuture)pendingTemplateFuts.putIfAbsent(U.maskNull(cacheCfg.getName()), fut);
 
         if (old != null)
             fut = old;
@@ -2787,7 +2785,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings("unchecked")
     public <K, V> IgniteCacheProxy<K, V> jcache(@Nullable String name) {
-        IgniteCacheProxy<K, V> cache = (IgniteCacheProxy<K, V>)jCacheProxies.get(maskNull(name));
+        IgniteCacheProxy<K, V> cache = (IgniteCacheProxy<K, V>)jCacheProxies.get(U.maskNull(name));
 
         if (cache == null)
             throw new IllegalArgumentException("Cache is not configured: " + name);
@@ -2829,7 +2827,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (log.isDebugEnabled())
             log.debug("Getting internal cache adapter: " + name);
 
-        return (GridCacheAdapter<K, V>)caches.get(maskNull(name));
+        return (GridCacheAdapter<K, V>)caches.get(U.maskNull(name));
     }
 
     /**
@@ -2852,7 +2850,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return {@code True} if specified cache is system, {@code false} otherwise.
      */
     public boolean systemCache(@Nullable String name) {
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(name));
+        DynamicCacheDescriptor desc = registeredCaches.get(U.maskNull(name));
 
         return desc != null && !desc.cacheType().userCache();
     }
@@ -3030,24 +3028,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param name Name to mask.
-     * @return Masked name.
-     */
-    private static String maskNull(String name) {
-        return name == null ? NULL_NAME : name;
-    }
-
-    /**
-     * @param name Name to unmask.
-     * @return Unmasked name.
-     */
-    @SuppressWarnings("StringEquality")
-    private static String unmaskNull(String name) {
-        // Intentional identity equality.
-        return name == NULL_NAME ? null : name;
-    }
-
-    /**
      *
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
@@ -3091,7 +3071,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         /** {@inheritDoc} */
         @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
             // Make sure to remove future before completion.
-            pendingFuts.remove(maskNull(cacheName), this);
+            pendingFuts.remove(U.maskNull(cacheName), this);
 
             return super.onDone(res, err);
         }
@@ -3133,7 +3113,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         /** {@inheritDoc} */
         @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
             // Make sure to remove future before completion.
-            pendingTemplateFuts.remove(maskNull(cacheName), this);
+            pendingTemplateFuts.remove(U.maskNull(cacheName), this);
 
             return super.onDone(res, err);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/121ddafe/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 46a23d6..08dea3b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -93,6 +93,9 @@ public abstract class IgniteUtils {
     /** Offset. */
     private static final int BYTE_ARRAY_DATA_OFFSET = UNSAFE.arrayBaseOffset(byte[].class);
 
+    /** Null cache name. */
+    private static final String NULL_NAME = U.id8(UUID.randomUUID());
+
     /** Sun-specific JDK constructor factory for objects that don't have empty constructor. */
     private static final Method CTOR_FACTORY;
 
@@ -902,8 +905,8 @@ public abstract class IgniteUtils {
         Runtime runtime = Runtime.getRuntime();
 
         X.println('<' + DEBUG_DATE_FMT.format(new Date(System.currentTimeMillis())) + "><DEBUG><" +
-            Thread.currentThread().getName() + "> Heap stats [free=" + runtime.freeMemory() / (1024 * 1024) +
-            "M, total=" + runtime.totalMemory() / (1024 * 1024) + "M]");
+                      Thread.currentThread().getName() + "> Heap stats [free=" + runtime.freeMemory() / (1024 * 1024) +
+                      "M, total=" + runtime.totalMemory() / (1024 * 1024) + "M]");
     }
 
     /**
@@ -3819,7 +3822,7 @@ public abstract class IgniteUtils {
             log.warning(compact(longMsg.toString()));
         else
             X.println("[" + SHORT_DATE_FMT.format(new java.util.Date()) + "] (wrn) " +
-                compact(shortMsg.toString()));
+                          compact(shortMsg.toString()));
     }
 
     /**
@@ -4024,6 +4027,28 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Masks null name of some Ignite component (ex. cache). Should be used if later the name will be unmasked with
+     * {@link #unmaskNull(String)}.
+     *
+     * @param name Name.
+     * @return Masked value if name is {@code null} or name as is.
+     */
+    public static String maskNull(@Nullable String name) {
+        return name == null ? NULL_NAME : name;
+    }
+
+    /**
+     * Unmasks name previously masked with {@link #maskNull(String)}.
+     *
+     * @param name Name.
+     * @return {@code null} if the name is masked or name as is.
+     */
+    @SuppressWarnings("StringEquality")
+    public static String unmaskNull(String name) {
+        return name == NULL_NAME ? null : NULL_NAME;
+    }
+
+    /**
      * Constructs JMX object name with given properties.
      * Map with ordered {@code groups} used for proper object name construction.
      *


[2/2] incubator-ignite git commit: ignite-1085: bugs fixing

Posted by sb...@apache.org.
ignite-1085: bugs fixing


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

Branch: refs/heads/ignite-1085
Commit: 7510aff0ed24a75ed4cd339f01c811d40aefe663
Parents: 121ddaf
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 9 12:44:34 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 9 12:44:34 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7510aff0/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 08dea3b..300a28b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -4045,7 +4045,7 @@ public abstract class IgniteUtils {
      */
     @SuppressWarnings("StringEquality")
     public static String unmaskNull(String name) {
-        return name == NULL_NAME ? null : NULL_NAME;
+        return name == NULL_NAME ? null : name;
     }
 
     /**