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 2015/08/26 10:00:03 UTC

[1/2] ignite git commit: IGNITE-1277: Enabled backups in IGFS data cache. Contributed by Ivan Veselovsky.

Repository: ignite
Updated Branches:
  refs/heads/master 76bc7d613 -> 4f3c9a2a1


IGNITE-1277: Enabled backups in IGFS data cache.
Contributed by Ivan Veselovsky.


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

Branch: refs/heads/master
Commit: 28d64d53143119fe64e5b2e8499bb2a978800efc
Parents: 203f00c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 10:59:54 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 10:59:54 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsProcessor.java |   8 -
 .../processors/igfs/IgfsAbstractSelfTest.java   |  34 +-
 .../igfs/IgfsBackupFailoverSelfTest.java        | 488 +++++++++++++++++++
 .../igfs/IgfsBackupsDualAsyncSelfTest.java      |  40 ++
 .../igfs/IgfsBackupsDualSyncSelfTest.java       |  40 ++
 .../igfs/IgfsBackupsPrimarySelfTest.java        |  40 ++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   7 +
 7 files changed, 643 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/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 af41ec4..f3522a8 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
@@ -311,14 +311,6 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
                         ", maxIgfsSpaceSize=" + maxSpaceSize + ']');
             }
 
-            if (dataCacheCfg.getCacheMode() == PARTITIONED) {
-                int backups = dataCacheCfg.getBackups();
-
-                if (backups != 0)
-                    throw new IgniteCheckedException("IGFS data cache cannot be used with backups (set backup count " +
-                        "to 0 and restart the grid): " + cfg.getDataCacheName());
-            }
-
             if (cfg.getMaxSpaceSize() == 0 && dataCacheCfg.getMemoryMode() == OFFHEAP_VALUES)
                 U.warn(log, "IGFS max space size is not specified but data cache values are stored off-heap (max " +
                     "space will be limited to 80% of max JVM heap size): " + cfg.getName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/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 a8a8957..85aaeb3 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
@@ -146,6 +146,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         this(mode, ONHEAP_TIERED);
     }
 
+    /**
+     * Constructor.
+     *
+     * @param mode
+     * @param memoryMode
+     */
     protected IgfsAbstractSelfTest(IgfsMode mode, CacheMemoryMode memoryMode) {
         assert mode != null && mode != PROXY;
 
@@ -155,7 +161,12 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
         dual = mode != PRIMARY;
     }
 
-    private static byte[] createChunk(int length) {
+    /**
+     *
+     * @param length
+     * @return
+     */
+    static byte[] createChunk(int length) {
         byte[] chunk = new byte[length];
 
         for (int i = 0; i < chunk.length; i++)
@@ -246,6 +257,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
 
+        prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
+
         cfg.setDiscoverySpi(discoSpi);
         cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
         cfg.setFileSystemConfiguration(igfsCfg);
@@ -257,6 +270,15 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     *
+     * @param dataCacheCfg
+     * @param metaCacheCfg
+     */
+    protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg, CacheConfiguration metaCacheCfg) {
+        // Noop
+    }
+
+    /**
      * Execute provided task in a separate thread.
      *
      * @param task Task to execute.
@@ -2263,7 +2285,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Data chunks.
      * @throws Exception If failed.
      */
-    protected void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize,
+    protected static void createFile(IgfsImpl igfs, IgfsPath file, boolean overwrite, long blockSize,
         @Nullable byte[]... chunks) throws Exception {
         IgfsOutputStream os = null;
 
@@ -2287,7 +2309,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param chunks Data chunks.
      * @throws Exception If failed.
      */
-    protected void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
+    protected static void appendFile(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
         throws Exception {
         IgfsOutputStream os = null;
 
@@ -2354,7 +2376,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param paths Paths.
      * @throws IgniteCheckedException If failed.
      */
-    protected void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
+    protected static void checkExist(IgfsImpl igfs, IgfsPath... paths) throws IgniteCheckedException {
         for (IgfsPath path : paths) {
             assert igfs.context().meta().fileId(path) != null : "Path doesn't exist [igfs=" + igfs.name() +
                 ", path=" + path + ']';
@@ -2463,7 +2485,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws IOException In case of IO exception.
      * @throws IgniteCheckedException In case of Grid exception.
      */
-    protected void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
+    protected static void checkFileContent(IgfsImpl igfs, IgfsPath file, @Nullable byte[]... chunks)
         throws IOException, IgniteCheckedException {
         if (chunks != null && chunks.length > 0) {
             IgfsInputStream is = null;
@@ -2562,7 +2584,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param paths Paths to group.
      * @return Paths as array.
      */
-    protected IgfsPath[] paths(IgfsPath... paths) {
+    protected static IgfsPath[] paths(IgfsPath... paths) {
         return paths;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
new file mode 100644
index 0000000..0162121
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupFailoverSelfTest.java
@@ -0,0 +1,488 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.*;
+
+/**
+ * Tests IGFS behavioral guarantees if some nodes on the cluster are synchronously or asynchronously stopped.
+ * The operations to check are read, write or both.
+ */
+public class IgfsBackupFailoverSelfTest extends IgfsCommonAbstractTest {
+    /** Directory. */
+    protected static final IgfsPath DIR = new IgfsPath("/dir");
+
+    /** Sub-directory. */
+    protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir");
+
+    /** Number of Ignite nodes used in failover tests. */
+    protected final int numIgfsNodes = 5;
+
+    /** Number of backup copies of data (aka replication). */
+    protected final int numBackups = numIgfsNodes - 1;
+
+    /** */
+    private final int fileSize = 16 * 1024;
+
+    /** */
+    private final int files = 500;
+
+    /** File block size. Use Very small blocks to ensure uniform data distribution among the nodes.. */
+    protected int igfsBlockSize = 31;
+
+    /** Affinity group size (see IgfsGroupDataBlocksKeyMapper). */
+    protected int affGrpSize = 1;
+
+    /** IGFS mode. */
+    protected IgfsMode igfsMode = IgfsMode.PRIMARY;
+
+    /** Node data structures. */
+    protected NodeFsData[] nodeDatas;
+
+    /**
+     * Structure to hold Ignite IGFS node data.
+     */
+    static class NodeFsData {
+        /**  */
+        int idx;
+
+        /**  */
+        Ignite ignite;
+
+        /**  */
+        IgfsImpl igfsImpl;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        nodeDatas = new NodeFsData[numIgfsNodes];
+
+        for (int i = 0; i<numIgfsNodes; i++) {
+            NodeFsData data = new NodeFsData();
+
+            data.idx = i;
+
+            data.ignite = startGridWithIgfs(getTestGridName(i), igfsMode, null);
+
+            data.igfsImpl = (IgfsImpl) data.ignite.fileSystem("igfs");
+
+            nodeDatas[i] = data;
+        }
+
+        // Ensure all the nodes are started and discovered each other:
+        checkTopology(numIgfsNodes);
+    }
+
+    /**
+     * Creates IPC configuration.
+     *
+     * @param port The port to use.
+     * @return The endpoint configuration.
+     */
+    protected IgfsIpcEndpointConfiguration createIgfsRestConfig(int port) {
+        IgfsIpcEndpointConfiguration cfg = new IgfsIpcEndpointConfiguration();
+
+        cfg.setType(IgfsIpcEndpointType.TCP);
+        cfg.setPort(port);
+
+        return cfg;
+    }
+
+    /**
+     * Start grid with IGFS.
+     *
+     * @param gridName Grid name.
+     * @param mode IGFS mode.
+     * @param secondaryFs Secondary file system (optional).
+     * @return Started grid instance.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithIgfs(String gridName, IgfsMode mode, @Nullable IgfsSecondaryFileSystem secondaryFs)
+        throws Exception {
+        final FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName("igfs");
+        igfsCfg.setBlockSize(igfsBlockSize);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setSecondaryFileSystem(secondaryFs);
+
+        CacheConfiguration<?,?> dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(affGrpSize));
+        dataCacheCfg.setBackups(numBackups);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName(gridName);
+
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        return startGrid(gridName, cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        G.stopAll(false);
+
+        Arrays.fill(nodeDatas, null);
+    }
+
+    /**
+     * Creates a chunk of data.
+     *
+     * @param len The chunk length
+     * @param j index to scramble into the data.
+     * @return The chunk.
+     */
+    static byte[] createChunk(int len, int j) {
+        byte[] bb = new byte[len];
+
+        for (int i = 0; i < bb.length; i++)
+            bb[i] = (byte)(i ^ j);
+
+        return bb;
+    }
+
+    /**
+     * Composes the path of the file.
+     */
+    private IgfsPath filePath(int j) {
+        return new IgfsPath(SUBDIR, "file" + j);
+    }
+
+    /**
+     * Checks correct data read *after* N-1 nodes are stopped.
+     *
+     * @throws Exception On error.
+     */
+    public void testReadFailoverAfterStopMultipleNodes() throws Exception {
+        final IgfsImpl igfs0 = nodeDatas[0].igfsImpl;
+
+        clear(igfs0);
+
+        IgfsAbstractSelfTest.create(igfs0, paths(DIR, SUBDIR), null);
+
+        // Create files through the 0th node:
+        for (int f=0; f<files; f++) {
+            final byte[] data = createChunk(fileSize, f);
+
+            createFile(igfs0, filePath(f), true, -1/*block size unused*/, data);
+        }
+
+        // Check files:
+        for (int f=0; f<files; f++) {
+            IgfsPath path = filePath(f);
+            byte[] data = createChunk(fileSize, f);
+
+            // Check through 0th node:
+            checkExist(igfs0, path);
+
+            checkFileContent(igfs0, path, data);
+
+            // Check the same file through other nodes:
+            for (int n=1; n<numIgfsNodes; n++) {
+                checkExist(nodeDatas[n].igfsImpl, path);
+
+                checkFileContent(nodeDatas[n].igfsImpl, path, data);
+            }
+        }
+
+        // Now stop all the nodes but the 1st:
+        for (int n=1; n<numIgfsNodes; n++)
+            stopGrid(n);
+
+        // Check files again:
+        for (int f=0; f<files; f++) {
+            IgfsPath path = filePath(f);
+
+            byte[] data = createChunk(fileSize, f);
+
+            // Check through 0th node:
+            checkExist(igfs0, path);
+
+            checkFileContent(igfs0, path, data);
+        }
+    }
+
+    /**
+     * Checks correct data read *while* N-1 nodes are being concurrently stopped.
+     *
+     * @throws Exception On error.
+     */
+    public void testReadFailoverWhileStoppingMultipleNodes() throws Exception {
+        final IgfsImpl igfs0 = nodeDatas[0].igfsImpl;
+
+        clear(igfs0);
+
+        IgfsAbstractSelfTest.create(igfs0, paths(DIR, SUBDIR), null);
+
+        // Create files:
+        for (int f = 0; f < files; f++) {
+            final byte[] data = createChunk(fileSize, f);
+
+            createFile(igfs0, filePath(f), true, -1/*block size unused*/, data);
+        }
+
+        // Check files:
+        for (int f = 0; f < files; f++) {
+            IgfsPath path = filePath(f);
+            byte[] data = createChunk(fileSize, f);
+
+            // Check through 1st node:
+            checkExist(igfs0, path);
+
+            checkFileContent(igfs0, path, data);
+
+            // Check the same file through other nodes:
+            for (int n = 1; n < numIgfsNodes; n++) {
+                checkExist(nodeDatas[n].igfsImpl, path);
+
+                checkFileContent(nodeDatas[n].igfsImpl, path, data);
+            }
+        }
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        GridTestUtils.runMultiThreadedAsync(new Callable() {
+            @Override
+            public Object call() throws Exception {
+                Thread.sleep(1_000); // Some delay to ensure read is in progress.
+
+                // Now stop all the nodes but the 1st:
+                for (int n = 1; n < numIgfsNodes; n++) {
+                    stopGrid(n);
+
+                    X.println("grid " + n + " stopped.");
+                }
+
+                Thread.sleep(1_000);
+
+                stop.set(true);
+
+                return null;
+            }
+        }, 1, "igfs-node-stopper");
+
+        // Read the files repeatedly, while the nodes are being stopped:
+        while (!stop.get()) {
+            // Check files while the nodes are being stopped:
+            for (int f = 0; f < files; f++) {
+                IgfsPath path = filePath(f);
+
+                byte[] data = createChunk(fileSize, f);
+
+                // Check through 1st node:
+                checkExist(igfs0, path);
+
+                checkFileContent(igfs0, path, data);
+            }
+        }
+    }
+
+    /**
+     * Checks possibility to append the data to files *after* N-1 nodes are stopped.
+     * First, some data written to files.
+     * After that N-1 nodes are stopped.
+     * Then data are attempted to append to the streams opened before the nodes stop.
+     * If failed, the streams are attempted to reopen and the files are attempted to append.
+     * After that the read operation is performed to check data correctness.
+     *
+     * The test is temporarily disabled due to issues .... .
+     *
+     * @throws Exception On error.
+     */
+    public void testWriteFailoverAfterStopMultipleNodes() throws Exception {
+        final IgfsImpl igfs0 = nodeDatas[0].igfsImpl;
+
+        clear(igfs0);
+
+        IgfsAbstractSelfTest.create(igfs0, paths(DIR, SUBDIR), null);
+
+        final IgfsOutputStream[] outStreams = new IgfsOutputStream[files];
+
+        // Create files:
+        for (int f = 0; f < files; f++) {
+            final byte[] data = createChunk(fileSize, f);
+
+            IgfsOutputStream os = null;
+
+            try {
+                os = igfs0.create(filePath(f), 256, true, null, 0, -1, null);
+
+                assert os != null;
+
+                writeFileChunks(os, data);
+            }
+            finally {
+                if (os != null)
+                    os.flush();
+            }
+
+            outStreams[f] = os;
+
+            X.println("write #1 completed: " + f);
+        }
+
+        // Now stop all the nodes but the 1st:
+        for (int n = 1; n < numIgfsNodes; n++) {
+            stopGrid(n);
+
+            X.println("#### grid " + n + " stopped.");
+        }
+
+        // Create files:
+        for (int f0 = 0; f0 < files; f0++) {
+            final IgfsOutputStream os = outStreams[f0];
+
+            assert os != null;
+
+            final int f = f0;
+
+            int att = doWithRetries(2, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgfsOutputStream ios = os;
+
+                    try {
+                        writeChunks0(igfs0, ios, f);
+                    }
+                    catch (IOException ioe) {
+                        log().warning("Attempt to append the data to existing stream failed: ", ioe);
+
+                        ios = igfs0.append(filePath(f), false);
+
+                        assert ios != null;
+
+                        writeChunks0(igfs0, ios, f);
+                    }
+
+                    return null;
+                }
+            });
+
+            X.println("write #2 completed: " + f0 + " in " + att + " attempts.");
+        }
+
+        // Check files:
+        for (int f = 0; f < files; f++) {
+            IgfsPath path = filePath(f);
+
+            byte[] data = createChunk(fileSize, f);
+
+            // Check through 1st node:
+            checkExist(igfs0, path);
+
+            assertEquals("File length mismatch.", data.length * 2, igfs0.size(path));
+
+            checkFileContent(igfs0, path, data, data);
+
+            X.println("Read test completed: " + f);
+        }
+    }
+
+    /**
+     * Writes data to the file of the specified index and closes the output stream.
+     *
+     * @param igfs0 IGFS.
+     * @param ios The output stream
+     * @param fileIdx Th eindex of the file.
+     * @throws IOException On error.
+     */
+    void writeChunks0(IgfsEx igfs0, IgfsOutputStream ios, int fileIdx) throws IOException {
+        try {
+            byte[] data = createChunk(fileSize, fileIdx);
+
+            writeFileChunks(ios, data);
+        }
+        finally {
+            ios.flush();
+
+            U.closeQuiet(ios);
+
+            awaitFileClose(igfs0.asSecondary(), filePath(fileIdx));
+        }
+    }
+
+    /**
+     * Performs an operation with retries.
+     *
+     * @param attempts The maximum number of attempts.
+     * @param clo The closure to execute.
+     * @throws Exception On error.
+     */
+    protected static int doWithRetries(int attempts, Callable<Void> clo) throws Exception {
+        int attemptCnt = 0;
+
+        while (true) {
+            try {
+                attemptCnt++;
+
+                clo.call();
+
+                return attemptCnt;
+            }
+            catch (Exception e) {
+                if (attemptCnt >= attempts)
+                    throw e;
+                else
+                    X.println("Failed to execute closure in " + attempts + " attempts.");
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 20 * 60 * 1000;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualAsyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualAsyncSelfTest.java
new file mode 100644
index 0000000..3c042d6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualAsyncSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.configuration.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+
+/**
+ * Tests for DUAL_ASYNC mode.
+ */
+public class IgfsBackupsDualAsyncSelfTest extends IgfsDualAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsBackupsDualAsyncSelfTest() {
+        super(DUAL_ASYNC);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg,
+        CacheConfiguration metaCacheCfg) {
+        dataCacheCfg.setBackups(1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualSyncSelfTest.java
new file mode 100644
index 0000000..5d7fdaf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsDualSyncSelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.configuration.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+
+/**
+ * Tests for DUAL_SYNC mode.
+ */
+public class IgfsBackupsDualSyncSelfTest extends IgfsDualAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsBackupsDualSyncSelfTest() {
+        super(DUAL_SYNC);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg,
+        CacheConfiguration metaCacheCfg) {
+        dataCacheCfg.setBackups(1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsPrimarySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsPrimarySelfTest.java
new file mode 100644
index 0000000..c4958a8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBackupsPrimarySelfTest.java
@@ -0,0 +1,40 @@
+/*
+ * 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.configuration.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+
+/**
+ * Tests for PRIMARY mode.
+ */
+public class IgfsBackupsPrimarySelfTest extends IgfsAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsBackupsPrimarySelfTest() {
+        super(PRIMARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void prepareCacheConfigurations(CacheConfiguration dataCacheCfg,
+            CacheConfiguration metaCacheCfg) {
+        dataCacheCfg.setBackups(1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/28d64d53/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 f8dac3d..73e6a03 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
@@ -79,6 +79,13 @@ public class IgniteIgfsTestSuite extends TestSuite {
 
         suite.addTestSuite(IgfsStartCacheTest.class);
 
+        suite.addTestSuite(IgfsBackupsPrimarySelfTest.class);
+        suite.addTestSuite(IgfsBackupsDualSyncSelfTest.class);
+        suite.addTestSuite(IgfsBackupsDualAsyncSelfTest.class);
+
+        // TODO: Enable when IGFS failover is fixed.
+        //suite.addTestSuite(IgfsBackupFailoverSelfTest.class);
+
         return suite;
     }
 }


[2/2] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/master
Commit: 4f3c9a2a1b54c6a61c790b668292a0ed1c33140e
Parents: 28d64d5 76bc7d6
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Aug 26 11:00:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Aug 26 11:00:44 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheManyAsyncOperationsTest.java     | 107 +++++++++++++++++++
 1 file changed, 107 insertions(+)
----------------------------------------------------------------------