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

[1/2] incubator-ignite git commit: #ignite-841: add tests.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-841 [created] 2d2fd56bd


#ignite-841: add tests.


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

Branch: refs/heads/ignite-841
Commit: b2184b3de433bda24f73d94d4d39212593858d11
Parents: 54f9492
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue May 5 17:06:52 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue May 5 17:06:52 2015 +0300

----------------------------------------------------------------------
 .../igfs/IgfsClientCacheSelfTest.java           | 108 +++++++++++++++++++
 1 file changed, 108 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b2184b3d/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
new file mode 100644
index 0000000..9ca69b3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for igfs with incorrect configuration.
+ */
+public class IgfsClientCacheSelfTest  extends GridCommonAbstractTest {
+    /** Meta-information cache name. */
+    private static final String META_CACHE_NAME = "meta";
+
+    /** Data cache name. */
+    private static final String DATA_CACHE_NAME = null;
+
+    /** Regular cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(cacheConfiguration(META_CACHE_NAME), cacheConfiguration(DATA_CACHE_NAME),
+            cacheConfiguration(CACHE_NAME));
+
+        if (!gridName.equals(getTestGridName(0)))
+            cfg.setClientMode(true);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setMetaCacheName(META_CACHE_NAME);
+        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
+        igfsCfg.setName("igfs");
+
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    protected CacheConfiguration cacheConfiguration(String cacheName) {
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName(cacheName);
+
+        if (META_CACHE_NAME.equals(cacheName))
+            cacheCfg.setCacheMode(REPLICATED);
+        else {
+            cacheCfg.setCacheMode(PARTITIONED);
+            cacheCfg.setNearConfiguration(null);
+
+            cacheCfg.setBackups(0);
+            cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+        }
+
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        return cacheCfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartIgfs() throws Exception {
+        startGrids(2);
+    }
+}


[2/2] incubator-ignite git commit: #ignite-841: fix test.

Posted by sb...@apache.org.
#ignite-841: fix test.


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

Branch: refs/heads/ignite-841
Commit: 2d2fd56bd9ab132e28347fb296101f2b081c8090
Parents: b2184b3
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue May 5 18:05:58 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue May 5 18:05:58 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    | 78 +++++++++++++++-----
 .../processors/igfs/IgfsDataManager.java        |  4 +-
 .../processors/igfs/IgfsMetaManager.java        |  2 +-
 .../igfs/IgfsClientCacheSelfTest.java           | 10 ++-
 4 files changed, 69 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d2fd56b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index c0026ab..f5371de 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
@@ -2447,6 +2447,30 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param name Cache name.
+     * @param <K> type of keys.
+     * @param <V> type of values.
+     * @return Cache instance for given name.
+     */
+    @SuppressWarnings("unchecked")
+    public <K, V> IgniteInternalCache<K, V> getOrStartCache(@Nullable String name) {
+        try {
+            if (log.isDebugEnabled())
+                log.debug("Getting cache for name: " + name);
+
+            IgniteCache<K, V> jcache = (IgniteCache<K, V>)jCacheProxies.get(maskNull(name));
+
+            if (jcache == null)
+                jcache = startJCache(name, true);
+
+            return jcache == null ? null : ((IgniteCacheProxy<K, V>)jcache).internalProxy();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
+    }
+
+    /**
      * @return All configured cache instances.
      */
     public Collection<IgniteInternalCache<?, ?>> caches() {
@@ -2553,37 +2577,53 @@ 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) {
-            if (desc == null || desc.cancelled()) {
-                if (failIfNotStarted)
-                    throw new IllegalArgumentException("Cache is not started: " + cacheName);
+        if (cache == null)
+           cache = startJCache(cacheName, failIfNotStarted);
 
-                return null;
-            }
+        return cache;
+    }
 
-            DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
+    /**
+     * @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 IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
+        String masked = maskNull(cacheName);
 
-            req.cacheName(cacheName);
+        DynamicCacheDescriptor desc = registeredCaches.get(masked);
 
-            req.deploymentId(desc.deploymentId());
+        if (desc == null || desc.cancelled()) {
+            if (failIfNotStarted)
+                throw new IllegalArgumentException("Cache is not started: " + cacheName);
 
-            CacheConfiguration cfg = new CacheConfiguration(desc.cacheConfiguration());
+            return null;
+        }
 
-            cfg.setNearConfiguration(null);
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
 
-            req.startCacheConfiguration(cfg);
+        req.cacheName(cacheName);
 
-            req.cacheType(desc.cacheType());
+        req.deploymentId(desc.deploymentId());
 
-            req.clientStartOnly(true);
+        CacheConfiguration cfg = new CacheConfiguration(desc.cacheConfiguration());
 
-            F.first(initiateCacheChanges(F.asList(req))).get();
+        cfg.setNearConfiguration(null);
 
-            cache = (IgniteCache<K, V>)jCacheProxies.get(masked);
+        req.startCacheConfiguration(cfg);
 
-            if (cache == null && failIfNotStarted)
-                throw new IllegalArgumentException("Cache is not started: " + cacheName);
-        }
+        req.cacheType(desc.cacheType());
+
+        req.clientStartOnly(true);
+
+        F.first(initiateCacheChanges(F.asList(req))).get();
+
+        IgniteCache cache = jCacheProxies.get(masked);
+
+        if (cache == null && failIfNotStarted)
+            throw new IllegalArgumentException("Cache is not started: " + cacheName);
 
         return cache;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d2fd56b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index 319b696..03bd519 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -202,8 +202,8 @@ public class IgfsDataManager extends IgfsManager {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
-        dataCachePrj = igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName());
-        dataCache = igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName());
+        dataCachePrj = igfsCtx.kernalContext().cache().getOrStartCache(igfsCtx.configuration().getDataCacheName());
+        dataCache = igfsCtx.kernalContext().cache().getOrStartCache(igfsCtx.configuration().getDataCacheName());
 
         metrics = igfsCtx.igfs().localMetrics();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d2fd56b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 35ca8bb..e33e0d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -108,7 +108,7 @@ public class IgfsMetaManager extends IgfsManager {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
-        metaCache = igfsCtx.kernalContext().cache().cache(cfg.getMetaCacheName());
+        metaCache = igfsCtx.kernalContext().cache().getOrStartCache(cfg.getMetaCacheName());
 
         assert metaCache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2d2fd56b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
index 9ca69b3..8e85d24 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -31,6 +32,9 @@ import static org.apache.ignite.cache.CacheMode.*;
  * Test for igfs with incorrect configuration.
  */
 public class IgfsClientCacheSelfTest  extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Meta-information cache name. */
     private static final String META_CACHE_NAME = "meta";
 
@@ -50,11 +54,11 @@ public class IgfsClientCacheSelfTest  extends GridCommonAbstractTest {
         if (!gridName.equals(getTestGridName(0)))
             cfg.setClientMode(true);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        disco.setIpFinder(IP_FINDER);
 
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(disco);
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();