Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 9790C1852B for ; Fri, 28 Aug 2015 12:49:20 +0000 (UTC) Received: (qmail 89342 invoked by uid 500); 28 Aug 2015 12:49:20 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 89237 invoked by uid 500); 28 Aug 2015 12:49:20 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 88888 invoked by uid 99); 28 Aug 2015 12:49:20 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 28 Aug 2015 12:49:20 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 1566AE7EAF; Fri, 28 Aug 2015 12:49:20 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Fri, 28 Aug 2015 12:49:31 -0000 Message-Id: <3c2f4103ccb8455ba114040b30f3e26a@git.apache.org> In-Reply-To: <03d00f7f08c642578bff5db4caeea5e6@git.apache.org> References: <03d00f7f08c642578bff5db4caeea5e6@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [13/41] ignite git commit: IGNITE-1277: Enabled backups in IGFS data cache. Contributed by Ivan Veselovsky. 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/ignite-1093 Commit: 28d64d53143119fe64e5b2e8499bb2a978800efc Parents: 203f00c Author: vozerov-gridgain Authored: Wed Aug 26 10:59:54 2015 +0300 Committer: vozerov-gridgain 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 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() { + @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 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; } }