You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/05/08 04:58:25 UTC

[29/36] incubator-ignite git commit: #ignite-841: NullPointerException at IgfsMetaManager.onKernalStart0 (IgfsMetaManager.java:115).

#ignite-841: NullPointerException at IgfsMetaManager.onKernalStart0 (IgfsMetaManager.java:115).


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

Branch: refs/heads/ignite-sprint-5
Commit: a452dac26c47ac10586f1f49bc26f03b6b5e6fd4
Parents: d4908f2
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 6 13:44:30 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 6 13:44:30 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  72 +++++++---
 .../processors/igfs/IgfsDataManager.java        |   3 +
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../igfs/IgfsClientCacheSelfTest.java           | 132 +++++++++++++++++++
 .../processors/igfs/IgfsStreamsSelfTest.java    |   2 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   2 +
 6 files changed, 192 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a452dac2/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 2b9a821..0e1a9c2 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
@@ -2452,6 +2452,24 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param name Cache name.
+     * @return Cache instance for given name.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public <K, V> IgniteInternalCache<K, V> getOrStartCache(@Nullable String name) throws IgniteCheckedException {
+        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();
+    }
+
+    /**
      * @return All configured cache instances.
      */
     public Collection<IgniteInternalCache<?, ?>> caches() {
@@ -2558,37 +2576,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/a452dac2/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..aa6427d 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,7 +202,10 @@ public class IgfsDataManager extends IgfsManager {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
+        igfsCtx.kernalContext().cache().getOrStartCache(igfsCtx.configuration().getDataCacheName());
         dataCachePrj = igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName());
+
+        igfsCtx.kernalContext().cache().getOrStartCache(igfsCtx.configuration().getDataCacheName());
         dataCache = igfsCtx.kernalContext().cache().internalCache(igfsCtx.configuration().getDataCacheName());
 
         metrics = igfsCtx.igfs().localMetrics();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a452dac2/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/a452dac2/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..d983302
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.Ignite;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.internal.util.typedef.G;
+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 static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for igfs with nodes in client mode (see {@link IgniteConfiguration#setClientMode(boolean)}.
+ */
+public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** 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";
+
+    /**
+     * Constructor.
+     */
+    public IgfsClientCacheSelfTest() {
+        super(IgfsMode.PRIMARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        igfsSecondaryFileSystem = createSecondaryFileSystemStack();
+
+        Ignite ignite1 = G.start(getConfiguration(getTestGridName(1)));
+
+        igfs = (IgfsImpl) ignite1.fileSystem("igfs");
+    }
+
+    /**{@inheritDoc} */
+    protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
+        Ignite igniteSecondary = G.start(getConfiguration(getTestGridName(0)));
+
+        IgfsEx secondaryIgfsImpl = (IgfsEx)igniteSecondary.fileSystem("igfs");
+
+        igfsSecondary = new IgfsExUniversalFileSystemAdapter(secondaryIgfsImpl);
+
+        return secondaryIgfsImpl.asSecondary();
+    }
+
+    /**
+     *
+     * @param gridName Grid name.
+     * @return Ignite configuration.
+     * @throws Exception If failed.
+     */
+    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 disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setMetaCacheName(META_CACHE_NAME);
+        igfsCfg.setDataCacheName(DATA_CACHE_NAME);
+        igfsCfg.setName("igfs");
+
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @return Cache configuration.
+     */
+    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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a452dac2/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
index 57b1010..d377560 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsStreamsSelfTest.java
@@ -154,7 +154,7 @@ public class IgfsStreamsSelfTest extends IgfsCommonAbstractTest {
      */
     public void testConfiguration() throws IgniteCheckedException {
         IgniteInternalCache metaCache = getFieldValue(fs, "meta", "metaCache");
-        GridCacheAdapter dataCache = getFieldValue(fs, "data", "dataCache");
+        IgniteInternalCache dataCache = getFieldValue(fs, "data", "dataCache");
 
         assertNotNull(metaCache);
         assertEquals(META_CACHE_NAME, metaCache.name());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a452dac2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index 11e22b1..4f3178e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -59,6 +59,8 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
         suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
 
+        suite.addTest(new TestSuite(IgfsClientCacheSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsModeResolverSelfTest.class));
 
         suite.addTestSuite(IgfsFragmentizerSelfTest.class);