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 533D017F01 for ; Thu, 8 Oct 2015 03:01:41 +0000 (UTC) Received: (qmail 96841 invoked by uid 500); 8 Oct 2015 03:01:41 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 96660 invoked by uid 500); 8 Oct 2015 03:01:41 -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 96631 invoked by uid 99); 8 Oct 2015 03:01:41 -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; Thu, 08 Oct 2015 03:01:41 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 56E38E0BAF; Thu, 8 Oct 2015 03:01:40 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: anovikov@apache.org To: commits@ignite.apache.org Date: Thu, 08 Oct 2015 03:01:55 -0000 Message-Id: In-Reply-To: <4b8e67c4fe354c01a0d4207d1d9dc7da@git.apache.org> References: <4b8e67c4fe354c01a0d4207d1d9dc7da@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [17/29] ignite git commit: IGNITE-1573: Fixed IGFS "mkdirs" concurrency problem. IGNITE-1573: Fixed IGFS "mkdirs" concurrency problem. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/a411f94d Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/a411f94d Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/a411f94d Branch: refs/heads/ignite-1168 Commit: a411f94d29b9073457ae2b31f89dd9f702e6bf53 Parents: fd091c8 Author: iveselovskiy Authored: Fri Oct 2 10:54:33 2015 +0300 Committer: vozerov-gridgain Committed: Fri Oct 2 10:54:33 2015 +0300 ---------------------------------------------------------------------- .../org/apache/ignite/IgniteFileSystem.java | 2 + .../internal/processors/igfs/IgfsFileInfo.java | 13 +- .../internal/processors/igfs/IgfsImpl.java | 64 +------ .../processors/igfs/IgfsMetaManager.java | 180 +++++++++++++++++- .../processors/igfs/IgfsAbstractSelfTest.java | 185 +++++++++++-------- 5 files changed, 300 insertions(+), 144 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java index a187a90..def2965 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteFileSystem.java @@ -417,6 +417,8 @@ public interface IgniteFileSystem extends IgniteAsyncSupport { /** * Creates directories under specified path with the specified properties. + * Note that the properties are applied only to created directories, but never + * updated for existing ones. * * @param path Path of directories chain to create. * @param props Metadata properties to set on created directories. http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java index 8564500..fa79d80 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java @@ -124,6 +124,16 @@ public final class IgfsFileInfo implements Externalizable { } /** + * Consturcts directory with random ID, provided listing and properties. + * + * @param listing Listing. + * @param props The properties to set for the new directory. + */ + IgfsFileInfo(@Nullable Map listing, @Nullable Map props) { + this(true/*dir*/, null, 0, 0, null, listing, props, null, false, System.currentTimeMillis(), false); + } + + /** * Constructs file info. * * @param blockSize Block size. @@ -194,7 +204,7 @@ public final class IgfsFileInfo implements Externalizable { * @param evictExclude Evict exclude flag. */ IgfsFileInfo(int blockSize, long len, boolean evictExclude, @Nullable Map props) { - this(blockSize == 0, // NB The contract is: (blockSize == null) <=> isDirectory() + this(blockSize == 0, // NB The contract is: (blockSize == 0) <=> isDirectory() null, blockSize, len, null, null, props, null, true, System.currentTimeMillis(), evictExclude); } @@ -495,6 +505,7 @@ public final class IgfsFileInfo implements Externalizable { } /** {@inheritDoc} */ + @SuppressWarnings("unchecked") @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { id = U.readGridUuid(in); blockSize = in.readInt(); http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java index d5ba95f..fa3a955 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java @@ -23,6 +23,7 @@ import java.net.URI; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; @@ -58,7 +59,6 @@ import org.apache.ignite.igfs.IgfsInvalidPathException; import org.apache.ignite.igfs.IgfsMetrics; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; -import org.apache.ignite.igfs.IgfsParentNotDirectoryException; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathAlreadyExistsException; import org.apache.ignite.igfs.IgfsPathIsDirectoryException; @@ -93,7 +93,6 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; -import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_CREATED; import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED; import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_RENAMED; import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_CLOSED_READ; @@ -769,71 +768,18 @@ public final class IgfsImpl implements IgfsEx { if (log.isDebugEnabled()) log.debug("Make directories: " + path); - Map props0 = props == null ? DFLT_DIR_META : props; + final Map props0 = props == null ? DFLT_DIR_META : new HashMap<>(props); IgfsMode mode = resolveMode(path); - if (mode != PRIMARY) { + if (mode == PRIMARY) + meta.mkdirs(path, props0); + else { assert mode == DUAL_SYNC || mode == DUAL_ASYNC; await(path); meta.mkdirsDual(secondaryFs, path, props0); - - return null; - } - - List ids = meta.fileIds(path); - List components = path.components(); - - assert ids.size() == components.size() + 1 : "Components doesn't contain ROOT element" + - " [ids=" + ids + ", components=" + components + ']'; - - IgniteUuid parentId = ROOT_ID; - - IgfsPath curPath = path.root(); - - for (int step = 0, size = components.size(); step < size; step++) { - IgniteUuid fileId = ids.get(step + 1); // Skip the first ROOT element. - - if (fileId == null) { - IgfsFileInfo fileInfo = new IgfsFileInfo(true, props0); // Create new directory. - - String fileName = components.get(step); // Get current component name. - - curPath = new IgfsPath(curPath, fileName); - - try { - // Fails only if parent is not a directory or if modified concurrently. - IgniteUuid oldId = meta.putIfAbsent(parentId, fileName, fileInfo); - - fileId = oldId == null ? fileInfo.id() : oldId; // Update node ID. - - if (oldId == null && evts.isRecordable(EVT_IGFS_DIR_CREATED)) - evts.record(new IgfsEvent(curPath, localNode(), EVT_IGFS_DIR_CREATED)); - } - catch (IgniteCheckedException e) { - if (log.isDebugEnabled()) - log.debug("Failed to create directory [path=" + path + ", parentId=" + parentId + - ", fileName=" + fileName + ", step=" + step + ", e=" + e.getMessage() + ']'); - - // Check directory with such name already exists. - IgfsFileInfo stored = meta.info(meta.fileId(parentId, fileName)); - - if (stored == null) - throw e; - - if (!stored.isDirectory()) - throw new IgfsParentNotDirectoryException("Failed to create directory (parent " + - "element is not a directory)"); - - fileId = stored.id(); // Update node ID. - } - } - - assert fileId != null; - - parentId = fileId; } return null; http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/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 bb6404c..927067a 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 @@ -51,6 +51,7 @@ import org.apache.ignite.igfs.IgfsConcurrentModificationException; import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException; import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsFile; +import org.apache.ignite.igfs.IgfsParentNotDirectoryException; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathAlreadyExistsException; import org.apache.ignite.igfs.IgfsPathIsDirectoryException; @@ -673,9 +674,10 @@ public class IgfsMetaManager extends IgfsManager { if (fileIds.contains(id) && !map.containsKey(id)) { IgfsFileInfo info = new IgfsFileInfo(id); - assert info.listing() != null; + IgfsFileInfo anotherInfo = id2InfoPrj.getAndPutIfAbsent(id, info); - id2InfoPrj.putIfAbsent(id, info); + if (anotherInfo != null) + info = anotherInfo; map.put(id, info); } @@ -1662,7 +1664,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Updated file info or {@code null} if such file ID not found. * @throws IgniteCheckedException If operation failed. */ - @Nullable private IgfsFileInfo updatePropertiesNonTx(@Nullable IgniteUuid parentId, IgniteUuid fileId, + @Nullable private IgfsFileInfo updatePropertiesNonTx(final @Nullable IgniteUuid parentId, final IgniteUuid fileId, String fileName, Map props) throws IgniteCheckedException { assert fileId != null; assert !F.isEmpty(props) : "Expects not-empty file's properties"; @@ -1672,8 +1674,8 @@ public class IgfsMetaManager extends IgfsManager { log.debug("Update file properties [fileId=" + fileId + ", props=" + props + ']'); try { - IgfsFileInfo oldInfo; - IgfsFileInfo parentInfo; + final IgfsFileInfo oldInfo; + final IgfsFileInfo parentInfo; // Lock file ID for this transaction. if (parentId == null) { @@ -1690,8 +1692,6 @@ public class IgfsMetaManager extends IgfsManager { return null; // Parent not found. } - assert validTxState(true); - if (oldInfo == null) return null; // File not found. @@ -1724,7 +1724,7 @@ public class IgfsMetaManager extends IgfsManager { if (parentId != null) { IgfsListingEntry entry = new IgfsListingEntry(newInfo); - assert metaCache.get(parentId) != null; + assert id2InfoPrj.get(parentId) != null; id2InfoPrj.invoke(parentId, new UpdateListing(fileName, entry, false)); } @@ -1874,6 +1874,170 @@ public class IgfsMetaManager extends IgfsManager { } /** + * Mkdirs implementation. + * + * @param path The path to create. + * @param props The properties to use for created directories. + * @return True iff a directory was created during the operation. + * @throws IgniteCheckedException If a non-directory file exists on the requested path, and in case of other errors. + */ + boolean mkdirs(final IgfsPath path, final Map props) throws IgniteCheckedException { + assert props != null; + assert validTxState(false); + + List components; + SortedSet idSet; + IgfsPath existingPath; + + while (true) { + if (busyLock.enterBusy()) { + try { + // Take the ids in *path* order out of transaction: + final List idList = fileIds(path); + + idSet = new TreeSet(PATH_ID_SORTING_COMPARATOR); + + idSet.add(ROOT_ID); + + components = path.components(); + + // Store all the non-null ids in the set & construct existing path in one loop: + existingPath = path.root(); + + assert idList.size() == components.size() + 1; + + // Find the lowermost existing id: + IgniteUuid parentId = ROOT_ID; + + for (int i = 1; i < idList.size(); i++) { + IgniteUuid id = idList.get(i); + + if (id == null) + break; + + parentId = id; + + boolean added = idSet.add(id); + + assert added; + + existingPath = new IgfsPath(existingPath, components.get(i - 1)); + } + + // Start TX. + IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ); + + try { + final Map lockedInfos = lockIds(idSet); + + // If the path was changed, we close the current Tx and repeat the procedure again + // starting from taking the path ids. + if (verifyPathIntegrity(existingPath, idList, lockedInfos)) { + // Locked path okay, trying to proceed with the remainder creation. + IgfsFileInfo parentInfo = lockedInfos.get(parentId); + + // Check only the lowermost directory in the existing directory chain + // because others are already checked in #verifyPathIntegrity() above. + if (!parentInfo.isDirectory()) + throw new IgfsParentNotDirectoryException("Failed to create directory (parent " + + "element is not a directory)"); + + if (idSet.size() == components.size() + 1) { + assert existingPath.equals(path); + assert lockedInfos.size() == idSet.size(); + + // The target directory already exists, nothing to do. + // (The fact that all the path consisns of directories is already checked above). + // Note that properties are not updated in this case. + return false; + } + + Map parentListing = parentInfo.listing(); + + String shortName = components.get(idSet.size() - 1); + + IgfsListingEntry entry = parentListing.get(shortName); + + if (entry == null) { + IgfsFileInfo childInfo = null; + + String childName = null; + + IgfsFileInfo newDirInfo; + + // This loop creates the missing directory chain from the bottom to the top: + for (int i = components.size() - 1; i >= idSet.size() - 1; i--) { + // Required entry does not exist. + // Create new directory info: + if (childName == null) { + assert childInfo == null; + + newDirInfo = new IgfsFileInfo(true, props); + } + else { + assert childInfo != null; + + newDirInfo = new IgfsFileInfo(Collections.singletonMap(childName, + new IgfsListingEntry(childInfo)), props); + } + + boolean put = id2InfoPrj.putIfAbsent(newDirInfo.id(), newDirInfo); + + assert put; // Because we used a new id that should be unique. + + childInfo = newDirInfo; + childName = components.get(i); + } + + // Now link the newly created directory chain to the lowermost existing parent: + id2InfoPrj.invoke(parentId, + new UpdateListing(childName, new IgfsListingEntry(childInfo), false)); + + // We're close to finish: + tx.commit(); + + break; + } + else { + // Another thread created file or directory with the same name. + if (!entry.isDirectory()) { + // Entry exists, and it is not a directory: + throw new IgfsParentNotDirectoryException("Failed to create directory (parent " + + "element is not a directory)"); + } + + // If this is a directory, we continue the repeat loop, + // because we cannot lock this directory without + // lock ordering rule violation. + } + } + } + finally { + tx.close(); + } + } + finally { + busyLock.leaveBusy(); + } + } + else + throw new IllegalStateException("Failed to mkdir because Grid is stopping. [path=" + path + ']'); + } // retry loop + + if (evts.isRecordable(EVT_IGFS_DIR_CREATED)) { + IgfsPath createdPath = existingPath; + + for (int i = idSet.size() - 1; i < components.size(); i++) { + createdPath = new IgfsPath(createdPath, components.get(i)); + + evts.record(new IgfsEvent(createdPath, locNode, EVT_IGFS_DIR_CREATED)); + } + } + + return true; + } + + /** * Set sampling flag. * * @param val Sampling flag state or {@code null} to clear sampling state and mark it as "not set". http://git-wip-us.apache.org/repos/asf/ignite/blob/a411f94d/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 7e73859..cc89fd1 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 @@ -57,6 +57,7 @@ import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration; import org.apache.ignite.igfs.IgfsIpcEndpointType; import org.apache.ignite.igfs.IgfsMode; import org.apache.ignite.igfs.IgfsOutputStream; +import org.apache.ignite.igfs.IgfsParentNotDirectoryException; import org.apache.ignite.igfs.IgfsPath; import org.apache.ignite.igfs.IgfsPathNotFoundException; import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem; @@ -101,12 +102,27 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { protected static final long BLOCK_SIZE = 32 * 1024 * 1024; /** Default repeat count. */ - protected static final int REPEAT_CNT = 5; // Diagnostic: ~100; Regression: 5 + protected static final int REPEAT_CNT = 10; /** Concurrent operations count. */ - protected static final int OPS_CNT = 16; // Diagnostic: ~160; Regression: 16 + protected static final int OPS_CNT = 32; - /** Seed. */ + /** Renames count. */ + protected static final int RENAME_CNT = OPS_CNT; + + /** Deletes count. */ + protected static final int DELETE_CNT = OPS_CNT; + + /** Updates count. */ + protected static final int UPDATE_CNT = OPS_CNT; + + /** Mkdirs count. */ + protected static final int MKDIRS_CNT = OPS_CNT; + + /** Create count. */ + protected static final int CREATE_CNT = OPS_CNT; + + /** Seed to generate random numbers. */ protected static final long SEED = System.currentTimeMillis(); /** Amount of blocks to prefetch. */ @@ -724,9 +740,46 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * * @throws Exception If failed. */ + @SuppressWarnings("ConstantConditions") public void testMkdirs() throws Exception { Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. + igfs.mkdirs(new IgfsPath("/x"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/x")); + + igfs.mkdirs(new IgfsPath("/k/l"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/k/l")); + + igfs.mkdirs(new IgfsPath("/x/y"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/x/y")); + + igfs.mkdirs(new IgfsPath("/a/b/c/d"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/a/b/c/d")); + + igfs.mkdirs(new IgfsPath("/a/b/c/d/e"), null); + checkExist(igfs, igfsSecondary, new IgfsPath("/a/b/c/d/e")); + + create(igfs, null, new IgfsPath[] { new IgfsPath("/d/f") }); // "f" is a file. + checkExist(igfs, igfsSecondary, new IgfsPath("/d/f")); + + try { + igfs.mkdirs(new IgfsPath("/d/f"), null); + + fail("IgfsParentNotDirectoryException expected."); + } + catch (IgfsParentNotDirectoryException ignore) { + // No-op. + } + + try { + igfs.mkdirs(new IgfsPath("/d/f/something/else"), null); + + fail("IgfsParentNotDirectoryException expected."); + } + catch (IgfsParentNotDirectoryException ignore) { + // No-op. + } + create(igfs, paths(DIR, SUBDIR), null); igfs.mkdirs(SUBSUBDIR, props); @@ -747,6 +800,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * * @throws Exception If failed. */ + @SuppressWarnings("ConstantConditions") public void testMkdirsParentRoot() throws Exception { Map props = properties(null, null, "0555"); // mkdirs command doesn't propagate user info. @@ -814,6 +868,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * * @throws Exception If failed. */ + @SuppressWarnings("ConstantConditions") public void testUpdate() throws Exception { Map props = properties("owner", "group", "0555"); @@ -832,6 +887,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * * @throws Exception If failed. */ + @SuppressWarnings("ConstantConditions") public void testUpdateParentRoot() throws Exception { Map props = properties("owner", "group", "0555"); @@ -936,6 +992,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { IgfsFile file = igfs.info(new IgfsPath("/")); + assert file != null; + Map props = file.properties(); assertEquals("moo", props.get("foo")); @@ -1619,8 +1677,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testConcurrentMkdirsDelete() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1541"); - for (int i = 0; i < REPEAT_CNT; i++) { final CyclicBarrier barrier = new CyclicBarrier(2); @@ -1900,14 +1956,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksRename() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, OPS_CNT, 0, 0, 0, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + checkDeadlocksRepeat(5, 2, 2, 2, RENAME_CNT, 0, 0, 0, 0); } /** @@ -1916,14 +1965,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksDelete() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, 0, OPS_CNT, 0, 0, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + checkDeadlocksRepeat(5, 2, 2, 2, 0, DELETE_CNT, 0, 0, 0); } /** @@ -1932,14 +1974,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksUpdate() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, 0, 0, OPS_CNT, 0, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, UPDATE_CNT, 0, 0); } /** @@ -1948,14 +1983,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksMkdirs() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, 0, 0, 0, OPS_CNT, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, 0, MKDIRS_CNT, 0); } /** @@ -1964,15 +1992,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksDeleteRename() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, - OPS_CNT, OPS_CNT, 0, 0, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + checkDeadlocksRepeat(5, 2, 2, 2, RENAME_CNT, DELETE_CNT, 0, 0, 0); } /** @@ -1980,18 +2000,17 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * * @throws Exception If failed. */ - public void testDeadlocksDeleteMkdirs() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1541"); + public void testDeadlocksDeleteMkdirsRename() throws Exception { + checkDeadlocksRepeat(5, 2, 2, 2, RENAME_CNT, DELETE_CNT, 0, MKDIRS_CNT, 0); + } - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, - 0, OPS_CNT, 0, OPS_CNT, 0); - } - finally { - clear(igfs, igfsSecondary); - } - } + /** + * Ensure that deadlocks do not occur during concurrent delete & rename operations. + * + * @throws Exception If failed. + */ + public void testDeadlocksDeleteMkdirs() throws Exception { + checkDeadlocksRepeat(5, 2, 2, 2, 0, DELETE_CNT, 0, MKDIRS_CNT, 0); } /** @@ -2000,14 +2019,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocksCreate() throws Exception { - for (int i = 0; i < REPEAT_CNT; i++) { - try { - checkDeadlocks(5, 2, 2, 2, 0, 0, 0, 0, OPS_CNT); - } - finally { - clear(igfs, igfsSecondary); - } - } + assert false : "https://issues.apache.org/jira/browse/IGNITE-1590"; + + checkDeadlocksRepeat(5, 2, 2, 2, 0, 0, 0, 0, CREATE_CNT); } /** @@ -2016,16 +2030,33 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ public void testDeadlocks() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-1541"); + assert false : "https://issues.apache.org/jira/browse/IGNITE-1590"; + checkDeadlocksRepeat(5, 2, 2, 2, RENAME_CNT, DELETE_CNT, UPDATE_CNT, MKDIRS_CNT, CREATE_CNT); + } + + /** + * Invokes {@link #checkDeadlocks(int, int, int, int, int, int, int, int, int)} for + * {@link #REPEAT_CNT} times. + * + * @param lvlCnt Total levels in folder hierarchy. + * @param childrenDirPerLvl How many children directories to create per level. + * @param childrenFilePerLvl How many children file to create per level. + * @param primaryLvlCnt How many levels will exist in the primary file system before check start. + * @param renCnt How many renames to perform. + * @param delCnt How many deletes to perform. + * @param updateCnt How many updates to perform. + * @param mkdirsCnt How many directory creations to perform. + * @param createCnt How many file creations to perform. + * @throws Exception If failed. + */ + private void checkDeadlocksRepeat(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl, + int primaryLvlCnt, int renCnt, int delCnt, + int updateCnt, int mkdirsCnt, int createCnt) throws Exception { for (int i = 0; i < REPEAT_CNT; i++) { try { - checkDeadlocks(5, 2, 2, 2, - OPS_CNT, // rename - OPS_CNT, // delete - OPS_CNT, // update - OPS_CNT, // mkdirs - OPS_CNT); // create + checkDeadlocks(lvlCnt, childrenDirPerLvl, childrenFilePerLvl, primaryLvlCnt, renCnt, delCnt, + updateCnt, mkdirsCnt, createCnt); } finally { clear(igfs, igfsSecondary); @@ -2050,9 +2081,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @throws Exception If failed. */ @SuppressWarnings("ConstantConditions") - public void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl, - int primaryLvlCnt, int renCnt, int delCnt, - int updateCnt, int mkdirsCnt, int createCnt) throws Exception { + private void checkDeadlocks(final int lvlCnt, final int childrenDirPerLvl, final int childrenFilePerLvl, + int primaryLvlCnt, int renCnt, int delCnt, int updateCnt, int mkdirsCnt, int createCnt) throws Exception { assert childrenDirPerLvl > 0; // First define file system structure. @@ -2096,7 +2126,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { // Now as we have all paths defined, plan operations on them. final Random rand = new Random(SEED); - int totalOpCnt = renCnt + delCnt + updateCnt + mkdirsCnt + createCnt; + final int totalOpCnt = renCnt + delCnt + updateCnt + mkdirsCnt + createCnt; + + if (totalOpCnt == 0) + throw new RuntimeException("Operations count is zero."); final CyclicBarrier barrier = new CyclicBarrier(totalOpCnt);