You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/03/28 08:34:10 UTC

[14/16] ignite git commit: IGNITE-2830: IGFS: Added multi-node tests. This closes #564.

IGNITE-2830: IGFS: Added multi-node tests. This closes #564.


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

Branch: refs/heads/ignite-gg-10994
Commit: b4f0dab089b0acae1542b7a397bfd3f7ce2e29e6
Parents: fb9e9b7
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri Mar 25 17:13:49 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 25 17:13:49 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsAbstractSelfTest.java   | 33 ++++++++++++++++----
 .../igfs/IgfsPrimaryMultiNodeSelfTest.java      | 28 +++++++++++++++++
 ...maryRelaxedConsistencyMultiNodeSelfTest.java | 28 +++++++++++++++++
 .../IgfsPrimaryRelaxedConsistencySelfTest.java  | 28 +++++++++++++++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        | 28 -----------------
 .../ignite/testsuites/IgniteIgfsTestSuite.java  | 12 +++++--
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |  3 +-
 7 files changed, 123 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 2f6c7bf..748a771 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -55,6 +55,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
@@ -189,6 +190,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Memory mode. */
     protected final CacheMemoryMode memoryMode;
 
+    /** Ignite nodes of cluster, excluding the secondary file system node, if any. */
+    protected Ignite[] nodes;
+
     static {
         PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
 
@@ -233,13 +237,20 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * @return Use optimzied marshaller flag.
+     * @return Use optimized marshaller flag.
      */
     protected boolean useOptimizedMarshaller() {
         return false;
     }
 
     /**
+     * @return Amount of nodes to start.
+     */
+    protected int nodeCount() {
+        return 1;
+    }
+
+    /**
      * Data chunk.
      *
      * @param len Length.
@@ -258,9 +269,17 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG);
+        TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+        nodes = new Ignite[nodeCount()];
+
+        for (int i = 0; i < nodes.length; i++) {
+            String nodeName = i == 0 ? "ignite" : "ignite" + i;
+
+            nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG, ipFinder);
+        }
 
-        igfs = (IgfsImpl) ignite.fileSystem("igfs");
+        igfs = (IgfsImpl) nodes[0].fileSystem("igfs");
     }
 
     /**
@@ -271,7 +290,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG);
+            SECONDARY_REST_CFG, new TcpDiscoveryVmIpFinder(true));
 
         IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
 
@@ -300,12 +319,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param mode IGFS mode.
      * @param secondaryFs Secondary file system (optional).
      * @param restCfg Rest configuration string (optional).
+     * @param ipFinder IP finder.
      * @return Started grid instance.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
     protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
+        TcpDiscoveryIpFinder ipFinder) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
@@ -347,7 +368,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        discoSpi.setIpFinder(ipFinder);
 
         prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java
new file mode 100644
index 0000000..1ee6d5a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryMultiNodeSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Multinode test for PRIMARY mode.
+ */
+public class IgfsPrimaryMultiNodeSelfTest extends IgfsPrimarySelfTest {
+    /** {@inheritDoc} */
+    @Override protected int nodeCount() {
+        return 4;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java
new file mode 100644
index 0000000..73a14c3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Tests for PRIMARY mode and relaxed consistency model.
+ */
+public class IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest extends IgfsPrimaryRelaxedConsistencySelfTest {
+    /** {@inheritDoc} */
+    @Override protected int nodeCount() {
+        return 4;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencySelfTest.java
new file mode 100644
index 0000000..0e8dd20
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedConsistencySelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * Tests for PRIMARY mode and relaxed consistency model.
+ */
+public class IgfsPrimaryRelaxedConsistencySelfTest extends IgfsPrimarySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean relaxedConsistency() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
deleted file mode 100644
index 6691df5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
+++ /dev/null
@@ -1,28 +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.
- */
-
-package org.apache.ignite.internal.processors.igfs;
-
-/**
- * Tests for PRIMARY mode and relaxed consistency model.
- */
-public class IgfsPrimaryRelaxedSelfTest extends IgfsPrimarySelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean relaxedConsistency() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/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 25c54e4..0047edb 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
@@ -38,11 +38,13 @@ import org.apache.ignite.internal.processors.igfs.IgfsMetaManagerSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsMetricsSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsModeResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsModesSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsOneClientNodeTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapTieredSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapValuesSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOptimziedMarshallerSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorValidationSelfTest;
@@ -70,10 +72,16 @@ public class IgniteIgfsTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite FS Test Suite For Platform Independent Tests");
 
         suite.addTest(new TestSuite(IgfsPrimarySelfTest.class));
-        suite.addTest(new TestSuite(IgfsPrimaryRelaxedSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryMultiNodeSelfTest.class));
+
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedConsistencySelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsPrimaryOptimziedMarshallerSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsPrimaryOffheapTieredSelfTest.class));
         suite.addTest(new TestSuite(IgfsPrimaryOffheapValuesSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
         suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index 10b1bcd..f4c1cb7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 
 import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.IGFS_SCHEME;
 import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH;
@@ -93,7 +94,7 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
      * @throws IOException On failure.
      */
     protected void startUnderlying() throws Exception {
-        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG, new TcpDiscoveryVmIpFinder(true));
     }
 
     /**