You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by av...@apache.org on 2016/11/24 13:24:15 UTC

[05/50] [abbrv] ignite git commit: IGNITE-3432: check data/meta cache names are different for different IGFS instances. This closes #1201

IGNITE-3432:  check data/meta cache names are different for different IGFS instances. This closes #1201


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

Branch: refs/heads/ignite-4242
Commit: 175da6b7e394dd76c27d5155ff98a5b2ef03bb9d
Parents: 6b78ad0
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Mon Nov 7 11:16:58 2016 +0500
Committer: tledkov-gridgain <tl...@gridgain.com>
Committed: Mon Nov 7 11:16:58 2016 +0500

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsProcessor.java | 15 ++++
 .../igfs/IgfsProcessorValidationSelfTest.java   | 30 ++++++++
 ...IpcEndpointRegistrationAbstractSelfTest.java | 76 ++++++++++++++++++--
 ...dpointRegistrationOnLinuxAndMacSelfTest.java | 11 +--
 4 files changed, 122 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/175da6b7/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 5c0e030..c72140d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cache.affinity.AffinityKeyMapper;
@@ -295,6 +296,9 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) throws IgniteCheckedException {
         Collection<String> cfgNames = new HashSet<>();
 
+        Collection<String> dataCacheNames = new HashSet<>();
+        Collection<String> metaCacheNames = new HashSet<>();
+
         for (FileSystemConfiguration cfg : cfgs) {
             String name = cfg.getName();
 
@@ -327,6 +331,14 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
             if (F.eq(cfg.getDataCacheName(), cfg.getMetaCacheName()))
                 throw new IgniteCheckedException("Cannot use same cache as both data and meta cache: " + cfg.getName());
 
+            if (dataCacheNames.contains(cfg.getDataCacheName()))
+                throw new IgniteCheckedException("Data cache names should be different for different IGFS instances: "
+                    + cfg.getName());
+
+            if (metaCacheNames.contains(cfg.getMetaCacheName()))
+                throw new IgniteCheckedException("Meta cache names should be different for different IGFS instances: "
+                    + cfg.getName());
+
             if (!(dataCacheCfg.getAffinityMapper() instanceof IgfsGroupDataBlocksKeyMapper))
                 throw new IgniteCheckedException("Invalid IGFS data cache configuration (key affinity mapper class should be " +
                     IgfsGroupDataBlocksKeyMapper.class.getSimpleName() + "): " + cfg);
@@ -360,6 +372,9 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
                     "secondaryFileSystem cannot be null when mode is not " + IgfsMode.PRIMARY);
             }
 
+            dataCacheNames.add(cfg.getDataCacheName());
+            metaCacheNames.add(cfg.getMetaCacheName());
+
             cfgNames.add(name);
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/175da6b7/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
index 97334da..28ec1da 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorValidationSelfTest.java
@@ -239,6 +239,36 @@ public class IgfsProcessorValidationSelfTest extends IgfsCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testLocalIfMetaCacheNameEquals() throws Exception {
+        g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
+
+        g1IgfsCfg1.setDataCacheName(dataCache1Name);
+        g1IgfsCfg1.setMetaCacheName(metaCache1Name);
+
+        g1IgfsCfg2.setDataCacheName(dataCache2Name);
+        g1IgfsCfg2.setMetaCacheName(metaCache1Name);
+
+        checkGridStartFails(g1Cfg, "Meta cache names should be different for different IGFS instances", true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLocalIfDataCacheNameEquals() throws Exception {
+        g1Cfg.setCacheConfiguration(concat(dataCaches(1024), metaCaches(), CacheConfiguration.class));
+
+        g1IgfsCfg1.setDataCacheName(dataCache1Name);
+        g1IgfsCfg1.setMetaCacheName(metaCache1Name);
+
+        g1IgfsCfg2.setDataCacheName(dataCache1Name);
+        g1IgfsCfg2.setMetaCacheName(metaCache2Name);
+
+        checkGridStartFails(g1Cfg, "Data cache names should be different for different IGFS instances", true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testRemoteIfDataBlockSizeDiffers() throws Exception {
         IgniteConfiguration g2Cfg = getConfiguration("g2");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/175da6b7/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
index 13a19fe..e58d503 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCheckedException;
@@ -48,7 +50,7 @@ import static org.apache.ignite.configuration.FileSystemConfiguration.DFLT_MGMT_
  */
 public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest extends IgfsCommonAbstractTest {
     /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+    protected static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     private static final AtomicInteger mgmtPort = new AtomicInteger(DFLT_MGMT_PORT);
 
@@ -80,12 +82,13 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
      * @throws Exception If failed.
      */
     public void testLoopbackEndpointsCustomHostRegistration() throws Exception {
-        IgniteConfiguration cfg = gridConfiguration();
+        IgniteConfiguration cfg = gridConfigurationManyIgfsCaches(2);
 
         cfg.setFileSystemConfiguration(
-            igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT, "127.0.0.1"),
+            igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT, "127.0.0.1",
+                "partitioned0", "replicated0"),
             igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1,
-                U.getLocalHost().getHostName()));
+                U.getLocalHost().getHostName(), "partitioned1", "replicated1"));
 
         G.start(cfg);
 
@@ -100,6 +103,7 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
      * Counts all registered IPC endpoints.
      *
      * @return Tuple2 where (tcp endpoints count, shmem endpoints count).
+     * @throws Exception If failed.
      */
     protected T2<Integer, Integer> checkRegisteredIpcEndpoints() throws Exception {
         GridKernalContext ctx = ((IgniteKernal)grid()).context();
@@ -151,15 +155,75 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
     }
 
     /**
+     * Creates base grid configuration.
+     *
+     * @param cacheCtn Caches count.
+     *
+     * @return Base grid configuration.
+     * @throws Exception In case of any error.
+     */
+    IgniteConfiguration gridConfigurationManyIgfsCaches(int cacheCtn) throws Exception {
+        IgniteConfiguration cfg = getConfiguration(getTestGridName());
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        List<CacheConfiguration> cachesCfg = new ArrayList<>();
+
+        for (int i = 0; i < cacheCtn; ++i) {
+            CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+            dataCacheCfg.setName("partitioned" + i);
+            dataCacheCfg.setCacheMode(CacheMode.PARTITIONED);
+            dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));
+            dataCacheCfg.setBackups(0);
+            dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+            metaCacheCfg.setName("replicated" + i);
+            metaCacheCfg.setCacheMode(CacheMode.REPLICATED);
+            metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            cachesCfg.add(dataCacheCfg);
+            cachesCfg.add(metaCacheCfg);
+        }
+
+        cfg.setCacheConfiguration(cachesCfg.toArray(new CacheConfiguration[cachesCfg.size()]));
+
+        return cfg;
+    }
+
+    /**
      * Creates test-purposed IgfsConfiguration.
      *
      * @param endPntType End point type.
      * @param endPntPort End point port.
      * @param endPntHost End point host.
      * @return test-purposed IgfsConfiguration.
+     * @throws IgniteCheckedException If failed.
      */
     protected FileSystemConfiguration igfsConfiguration(@Nullable IgfsIpcEndpointType endPntType,
         @Nullable Integer endPntPort, @Nullable String endPntHost) throws IgniteCheckedException {
+
+        return igfsConfiguration(endPntType, endPntPort, endPntHost, "partitioned", "replicated");
+    }
+
+    /**
+     * Creates test-purposed IgfsConfiguration.
+     *
+     * @param endPntType End point type.
+     * @param endPntPort End point port.
+     * @param endPntHost End point host.
+     * @param dataCacheName Data cache name.
+     * @param metaCacheName Meta cache name.
+     * @return test-purposed IgfsConfiguration.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected FileSystemConfiguration igfsConfiguration(@Nullable IgfsIpcEndpointType endPntType,
+        @Nullable Integer endPntPort, @Nullable String endPntHost, String dataCacheName, String metaCacheName) throws IgniteCheckedException {
         IgfsIpcEndpointConfiguration endPntCfg = null;
 
         if (endPntType != null) {
@@ -176,8 +240,8 @@ public abstract class IgfsServerManagerIpcEndpointRegistrationAbstractSelfTest e
 
         FileSystemConfiguration igfsConfiguration = new FileSystemConfiguration();
 
-        igfsConfiguration.setDataCacheName("partitioned");
-        igfsConfiguration.setMetaCacheName("replicated");
+        igfsConfiguration.setDataCacheName(dataCacheName);
+        igfsConfiguration.setMetaCacheName(metaCacheName);
         igfsConfiguration.setName("igfs" + UUID.randomUUID());
         igfsConfiguration.setManagementPort(mgmtPort.getAndIncrement());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/175da6b7/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
index 3d89874..030c852 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest.java
@@ -33,12 +33,15 @@ public class IgfsServerManagerIpcEndpointRegistrationOnLinuxAndMacSelfTest
      * @throws Exception If failed.
      */
     public void testLoopbackAndShmemEndpointsRegistration() throws Exception {
-        IgniteConfiguration cfg = gridConfiguration();
+        IgniteConfiguration cfg = gridConfigurationManyIgfsCaches(3);
 
         cfg.setFileSystemConfiguration(
-            igfsConfiguration(null, null, null), // Check null IPC endpoint config won't bring any hassles.
-            igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1, null),
-            igfsConfiguration(IgfsIpcEndpointType.SHMEM, IgfsIpcEndpointConfiguration.DFLT_PORT + 2, null));
+            // Check null IPC endpoint config won't bring any hassles.
+            igfsConfiguration(null, null, null, "partitioned0", "replicated0"),
+            igfsConfiguration(IgfsIpcEndpointType.TCP, IgfsIpcEndpointConfiguration.DFLT_PORT + 1, null,
+                "partitioned1", "replicated1"),
+            igfsConfiguration(IgfsIpcEndpointType.SHMEM, IgfsIpcEndpointConfiguration.DFLT_PORT + 2, null,
+                "partitioned2", "replicated2"));
 
         G.start(cfg);