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 B35A41906B for ; Tue, 22 Mar 2016 16:49:38 +0000 (UTC) Received: (qmail 60519 invoked by uid 500); 22 Mar 2016 16:49:38 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 60282 invoked by uid 500); 22 Mar 2016 16:49:38 -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 59214 invoked by uid 99); 22 Mar 2016 16:49:37 -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; Tue, 22 Mar 2016 16:49:37 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C9765DFC6F; Tue, 22 Mar 2016 16:49:37 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ntikhonov@apache.org To: commits@ignite.apache.org Date: Tue, 22 Mar 2016 16:50:12 -0000 Message-Id: <6fe5a4a39d884a3f8393d09cf593ce61@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [36/50] [abbrv] ignite git commit: IGNITE-2813: IGFS: Optimized metadata values splitting file and directory into separate classes. http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 d91b0bc..84e4dae 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 @@ -86,8 +86,6 @@ import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.CountDownLatch; -import static org.apache.ignite.internal.processors.igfs.IgfsFileInfo.builder; - /** * Cache based structure (meta data) manager. */ @@ -117,7 +115,7 @@ public class IgfsMetaManager extends IgfsManager { private CountDownLatch metaCacheStartLatch; /** File ID to file info projection. */ - private IgniteInternalCache id2InfoPrj; + private IgniteInternalCache id2InfoPrj; /** Predefined key for sampling mode value. */ private GridCacheInternal sampling; @@ -175,7 +173,7 @@ public class IgfsMetaManager extends IgfsManager { } }); - id2InfoPrj = (IgniteInternalCache)metaCache.cache(); + id2InfoPrj = (IgniteInternalCache)metaCache.cache(); locNode = igfsCtx.kernalContext().discovery().localNode(); @@ -440,17 +438,17 @@ public class IgfsMetaManager extends IgfsManager { * @return File info. * @throws IgniteCheckedException If failed. */ - @Nullable public IgfsFileInfo info(@Nullable IgniteUuid fileId) throws IgniteCheckedException { + @Nullable public IgfsEntryInfo info(@Nullable IgniteUuid fileId) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { if (fileId == null) return null; - IgfsFileInfo info = getInfo(fileId); + IgfsEntryInfo info = getInfo(fileId); // Force root ID always exist in cache. if (info == null && IgfsUtils.ROOT_ID.equals(fileId)) - info = createSystemEntryIfAbsent(fileId); + info = createSystemDirectoryIfAbsent(fileId); return info; } @@ -469,7 +467,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Files details. * @throws IgniteCheckedException If failed. */ - public Map infos(Collection fileIds) throws IgniteCheckedException { + public Map infos(Collection fileIds) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { validTxState(false); @@ -479,13 +477,13 @@ public class IgfsMetaManager extends IgfsManager { if (F.isEmpty(fileIds)) return Collections.emptyMap(); - Map map = getInfos(fileIds); + Map map = getInfos(fileIds); // Force root ID always exist in cache. if (fileIds.contains(IgfsUtils.ROOT_ID) && !map.containsKey(IgfsUtils.ROOT_ID)) { map = new GridLeanMap<>(map); - map.put(IgfsUtils.ROOT_ID, createSystemEntryIfAbsent(IgfsUtils.ROOT_ID)); + map.put(IgfsUtils.ROOT_ID, createSystemDirectoryIfAbsent(IgfsUtils.ROOT_ID)); } return map; @@ -506,7 +504,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Locked file info or {@code null} if file cannot be locked or doesn't exist. * @throws IgniteCheckedException If the file with such id does not exist, or on another failure. */ - public @Nullable IgfsFileInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException { + public @Nullable IgfsEntryInfo lock(IgniteUuid fileId, boolean delete) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { validTxState(false); @@ -517,7 +515,7 @@ public class IgfsMetaManager extends IgfsManager { try { // Lock file ID for this transaction. - IgfsFileInfo oldInfo = info(fileId); + IgfsEntryInfo oldInfo = info(fileId); if (oldInfo == null) return null; @@ -525,7 +523,7 @@ public class IgfsMetaManager extends IgfsManager { if (oldInfo.lockId() != null) return null; // The file is already locked, we cannot lock it. - IgfsFileInfo newInfo = invokeLock(fileId, delete); + IgfsEntryInfo newInfo = invokeLock(fileId, delete); tx.commit(); @@ -566,7 +564,7 @@ public class IgfsMetaManager extends IgfsManager { * @param modificationTime Modification time to write to file info. * @throws IgniteCheckedException If failed. */ - public void unlock(final IgfsFileInfo info, final long modificationTime) throws IgniteCheckedException { + public void unlock(final IgfsEntryInfo info, final long modificationTime) throws IgniteCheckedException { validTxState(false); assert info != null; @@ -589,7 +587,7 @@ public class IgfsMetaManager extends IgfsManager { IgniteUuid fileId = info.id(); // Lock file ID for this transaction. - IgfsFileInfo oldInfo = info(fileId); + IgfsEntryInfo oldInfo = info(fileId); if (oldInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to unlock file (file not " + @@ -628,7 +626,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Locked file details. Resulting map doesn't contain details for not-existent files. * @throws IgniteCheckedException If failed. */ - private Map lockIds(IgniteUuid... fileIds) throws IgniteCheckedException { + private Map lockIds(IgniteUuid... fileIds) throws IgniteCheckedException { validTxState(true); assert fileIds != null && fileIds.length > 0; @@ -668,7 +666,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Map with lock info. * @throws IgniteCheckedException If failed. */ - private Map lockIds(Collection fileIds) throws IgniteCheckedException { + private Map lockIds(Collection fileIds) throws IgniteCheckedException { assert isSorted(fileIds); validTxState(true); @@ -676,7 +674,7 @@ public class IgfsMetaManager extends IgfsManager { log.debug("Locking file ids: " + fileIds); // Lock files and get their infos. - Map map = getInfos(fileIds); + Map map = getInfos(fileIds); if (log.isDebugEnabled()) log.debug("Locked file ids: " + fileIds); @@ -684,7 +682,7 @@ public class IgfsMetaManager extends IgfsManager { for (IgniteUuid fileId : fileIds) { if (IgfsUtils.isRootOrTrashId(fileId)) { if (!map.containsKey(fileId)) - map.put(fileId, createSystemEntryIfAbsent(fileId)); + map.put(fileId, createSystemDirectoryIfAbsent(fileId)); } } @@ -699,13 +697,15 @@ public class IgfsMetaManager extends IgfsManager { * @return Value of created or existing system entry. * @throws IgniteCheckedException On error. */ - private IgfsFileInfo createSystemEntryIfAbsent(IgniteUuid id) + private IgfsEntryInfo createSystemDirectoryIfAbsent(IgniteUuid id) throws IgniteCheckedException { assert IgfsUtils.isRootOrTrashId(id); - IgfsFileInfo info = new IgfsFileInfo(id); + long time = System.currentTimeMillis(); + + IgfsEntryInfo info = IgfsUtils.createDirectory(id); - IgfsFileInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info); + IgfsEntryInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info); if (oldInfo != null) info = oldInfo; @@ -740,7 +740,7 @@ public class IgfsMetaManager extends IgfsManager { * @return First qualified file info. * @throws IgniteCheckedException If failed to get file for fragmentizer. */ - public IgfsFileInfo fileForFragmentizer(Collection exclude) throws IgniteCheckedException { + public IgfsEntryInfo fileForFragmentizer(Collection exclude) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { return fileForFragmentizer0(IgfsUtils.ROOT_ID, exclude); @@ -761,9 +761,9 @@ public class IgfsMetaManager extends IgfsManager { * @return First qualified file info. * @throws IgniteCheckedException If failed to get file for fragmentizer. */ - private IgfsFileInfo fileForFragmentizer0(IgniteUuid parentId, Collection exclude) + private IgfsEntryInfo fileForFragmentizer0(IgniteUuid parentId, Collection exclude) throws IgniteCheckedException { - IgfsFileInfo info = info(parentId); + IgfsEntryInfo info = info(parentId); // Check if file was concurrently deleted. if (info == null) @@ -775,7 +775,7 @@ public class IgfsMetaManager extends IgfsManager { for (IgfsListingEntry entry : listing.values()) { if (entry.isFile()) { - IgfsFileInfo fileInfo = info(entry.fileId()); + IgfsEntryInfo fileInfo = info(entry.fileId()); if (fileInfo != null) { if (!exclude.contains(fileInfo.id()) && @@ -785,7 +785,7 @@ public class IgfsMetaManager extends IgfsManager { } } else { - IgfsFileInfo fileInfo = fileForFragmentizer0(entry.fileId(), exclude); + IgfsEntryInfo fileInfo = fileForFragmentizer0(entry.fileId(), exclude); if (fileInfo != null) return fileInfo; @@ -807,7 +807,7 @@ public class IgfsMetaManager extends IgfsManager { throws IgniteCheckedException { assert fileId != null; - IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) : + IgfsEntryInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) : getInfo(fileId); return info == null ? Collections.emptyMap() : info.listing(); @@ -822,7 +822,7 @@ public class IgfsMetaManager extends IgfsManager { * @return File id already stored in meta cache or {@code null} if passed file info was stored. * @throws IgniteCheckedException If failed. */ - private IgniteUuid putIfAbsentNonTx(IgniteUuid parentId, String fileName, IgfsFileInfo newFileInfo) + private IgniteUuid putIfAbsentNonTx(IgniteUuid parentId, String fileName, IgfsEntryInfo newFileInfo) throws IgniteCheckedException { if (log.isDebugEnabled()) log.debug("Locking parent id [parentId=" + parentId + ", fileName=" + fileName + ", newFileInfo=" + @@ -831,7 +831,7 @@ public class IgfsMetaManager extends IgfsManager { validTxState(true); // Lock only parent file ID. - IgfsFileInfo parentInfo = info(parentId); + IgfsEntryInfo parentInfo = info(parentId); if (parentInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to lock parent directory (not found): " + @@ -858,7 +858,7 @@ public class IgfsMetaManager extends IgfsManager { * @return File info of renamed entry. * @throws IgniteCheckedException In case of exception. */ - public IgfsFileInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException { + public IgfsEntryInfo move(IgfsPath srcPath, IgfsPath dstPath) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { validTxState(false); @@ -902,7 +902,7 @@ public class IgfsMetaManager extends IgfsManager { try { // Obtain the locks. - final Map lockInfos = lockIds(lockIds); + final Map lockInfos = lockIds(lockIds); // Verify integrity of source and destination paths. if (!srcPathIds.verifyIntegrity(lockInfos)) @@ -914,7 +914,7 @@ public class IgfsMetaManager extends IgfsManager { "structure changed concurrently [src=" + srcPath + ", dst=" + dstPath + ']'); // Addiional check: is destination directory? - IgfsFileInfo dstParentInfo = lockInfos.get(dstPathIds.lastId()); + IgfsEntryInfo dstParentInfo = lockInfos.get(dstPathIds.lastId()); if (dstParentInfo.isFile()) throw new IgfsPathAlreadyExistsException("Failed to perform move because destination points " + @@ -927,9 +927,9 @@ public class IgfsMetaManager extends IgfsManager { ", dst=" + dstPath + ']'); // Actual move: remove from source parent and add to destination target. - IgfsFileInfo srcParentInfo = lockInfos.get(srcPathIds.lastParentId()); + IgfsEntryInfo srcParentInfo = lockInfos.get(srcPathIds.lastParentId()); - IgfsFileInfo srcInfo = lockInfos.get(srcPathIds.lastId()); + IgfsEntryInfo srcInfo = lockInfos.get(srcPathIds.lastId()); String srcName = srcPathIds.lastPart(); IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcName); @@ -940,7 +940,7 @@ public class IgfsMetaManager extends IgfsManager { IgfsPath newPath = new IgfsPath(dstPathIds.path(), dstName); // Set the new path to the info to simplify event creation: - return IgfsFileInfo.builder(srcInfo).path(newPath).build(); + return srcInfo.path(newPath); } finally { tx.close(); @@ -965,7 +965,7 @@ public class IgfsMetaManager extends IgfsManager { * @return verification result. */ private static boolean verifyPathIntegrity(IgfsPath path, List expIds, - Map infos) { + Map infos) { List pathParts = path.components(); assert pathParts.size() < expIds.size(); @@ -975,7 +975,7 @@ public class IgfsMetaManager extends IgfsManager { // If parent ID is null, it doesn't exist. if (parentId != null) { - IgfsFileInfo parentInfo = infos.get(parentId); + IgfsEntryInfo parentInfo = infos.get(parentId); // If parent info is null, it doesn't exist. if (parentInfo != null) { @@ -1019,9 +1019,9 @@ public class IgfsMetaManager extends IgfsManager { } // Lock file ID and parent IDs for this transaction. - Map infoMap = lockIds(srcParentId, fileId, destParentId); + Map infoMap = lockIds(srcParentId, fileId, destParentId); - IgfsFileInfo srcInfo = infoMap.get(srcParentId); + IgfsEntryInfo srcInfo = infoMap.get(srcParentId); if (srcInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to lock source directory (not found?)" + @@ -1030,7 +1030,7 @@ public class IgfsMetaManager extends IgfsManager { if (!srcInfo.isDirectory()) throw fsException(new IgfsPathIsNotDirectoryException("Source is not a directory: " + srcInfo)); - IgfsFileInfo destInfo = infoMap.get(destParentId); + IgfsEntryInfo destInfo = infoMap.get(destParentId); if (destInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to lock destination directory (not found?)" + @@ -1039,7 +1039,7 @@ public class IgfsMetaManager extends IgfsManager { if (!destInfo.isDirectory()) throw fsException(new IgfsPathIsNotDirectoryException("Destination is not a directory: " + destInfo)); - IgfsFileInfo fileInfo = infoMap.get(fileId); + IgfsEntryInfo fileInfo = infoMap.get(fileId); if (fileInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to lock target file (not found?) [fileId=" + @@ -1080,7 +1080,7 @@ public class IgfsMetaManager extends IgfsManager { try { // NB: We may lock root because its id is less than any other id: - final IgfsFileInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID); + final IgfsEntryInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID); assert rootInfo != null; @@ -1094,13 +1094,17 @@ public class IgfsMetaManager extends IgfsManager { // Construct new info and move locked entries from root to it. Map transferListing = new HashMap<>(rootListingMap); - IgfsFileInfo newInfo = new IgfsFileInfo(transferListing); + IgfsEntryInfo newInfo = IgfsUtils.createDirectory( + IgniteUuid.randomUuid(), + transferListing, + (Map)null + ); createNewEntry(newInfo, trashId, newInfo.id().toString()); // Remove listing entries from root. // Note that root directory properties and other attributes are preserved: - id2InfoPrj.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(null/*listing*/, rootInfo)); + id2InfoPrj.put(IgfsUtils.ROOT_ID, rootInfo.listing(null)); tx.commit(); @@ -1159,13 +1163,13 @@ public class IgfsMetaManager extends IgfsManager { try { // Lock participants. - Map lockInfos = lockIds(allIds); + Map lockInfos = lockIds(allIds); // Ensure that all participants are still in place. if (!pathIds.verifyIntegrity(lockInfos)) return null; - IgfsFileInfo victimInfo = lockInfos.get(victimId); + IgfsEntryInfo victimInfo = lockInfos.get(victimId); // Cannot delete non-empty directory if recursive flag is not set. if (!recursive && victimInfo.hasChildren()) @@ -1173,14 +1177,14 @@ public class IgfsMetaManager extends IgfsManager { "empty and recursive flag is not set)."); // Prepare trash data. - IgfsFileInfo trashInfo = lockInfos.get(trashId); + IgfsEntryInfo trashInfo = lockInfos.get(trashId); final String trashName = victimId.toString(); assert !trashInfo.hasChild(trashName) : "Failed to add file name into the " + "destination directory (file already exists) [destName=" + trashName + ']'; IgniteUuid parentId = pathIds.lastParentId(); - IgfsFileInfo parentInfo = lockInfos.get(parentId); + IgfsEntryInfo parentInfo = lockInfos.get(parentId); transferEntry(parentInfo.listing().get(victimName), parentId, victimName, trashId, trashName); @@ -1227,13 +1231,13 @@ public class IgfsMetaManager extends IgfsManager { // Handle special case when we deleting root directory. assert IgfsUtils.ROOT_ID.equals(id); - IgfsFileInfo rootInfo = getInfo(IgfsUtils.ROOT_ID); + IgfsEntryInfo rootInfo = getInfo(IgfsUtils.ROOT_ID); if (rootInfo == null) return null; // Root was never created. // Ensure trash directory existence. - createSystemEntryIfAbsent(trashId); + createSystemDirectoryIfAbsent(trashId); Map rootListing = rootInfo.listing(); @@ -1253,13 +1257,18 @@ public class IgfsMetaManager extends IgfsManager { transferListing.putAll(rootListing); - IgfsFileInfo newInfo = new IgfsFileInfo(transferListing); + IgfsEntryInfo newInfo = IgfsUtils.createDirectory( + IgniteUuid.randomUuid(), + transferListing, + (Map)null + ); createNewEntry(newInfo, trashId, newInfo.id().toString()); // Remove listing entries from root. for (Map.Entry entry : transferListing.entrySet()) - id2InfoPrj.invoke(IgfsUtils.ROOT_ID, new ListingRemoveProcessor(entry.getKey(), entry.getValue().fileId())); + id2InfoPrj.invoke(IgfsUtils.ROOT_ID, + new ListingRemoveProcessor(entry.getKey(), entry.getValue().fileId())); resId = newInfo.id(); } @@ -1268,7 +1277,7 @@ public class IgfsMetaManager extends IgfsManager { } else { // Ensure trash directory existence. - createSystemEntryIfAbsent(trashId); + createSystemDirectoryIfAbsent(trashId); moveNonTx(id, name, parentId, id.toString(), trashId); @@ -1310,9 +1319,9 @@ public class IgfsMetaManager extends IgfsManager { for (IgfsListingEntry childEntry : listing.values()) allIds[i++] = childEntry.fileId(); - Map locks = lockIds(allIds); + Map locks = lockIds(allIds); - IgfsFileInfo parentInfo = locks.get(parentId); + IgfsEntryInfo parentInfo = locks.get(parentId); // Ensure parent is still in place. if (parentInfo != null) { @@ -1327,7 +1336,7 @@ public class IgfsMetaManager extends IgfsManager { String childName = entry.getKey(); IgniteUuid childId = entry.getValue().fileId(); - IgfsFileInfo entryInfo = locks.get(childId); + IgfsEntryInfo entryInfo = locks.get(childId); if (entryInfo != null) { // File must be locked for deletion: @@ -1351,7 +1360,7 @@ public class IgfsMetaManager extends IgfsManager { } // Update parent listing. - id2InfoPrj.put(parentId, new IgfsFileInfo(newListing, parentInfo)); + id2InfoPrj.put(parentId, parentInfo.listing(newListing)); } tx.commit(); @@ -1391,9 +1400,9 @@ public class IgfsMetaManager extends IgfsManager { try { boolean res = false; - Map infos = lockIds(parentId, id); + Map infos = lockIds(parentId, id); - IgfsFileInfo victim = infos.get(id); + IgfsEntryInfo victim = infos.get(id); if (victim == null) return res; @@ -1403,7 +1412,7 @@ public class IgfsMetaManager extends IgfsManager { // Proceed only in case both parent and child exist. if (infos.containsKey(parentId) && infos.containsKey(id)) { - IgfsFileInfo parentInfo = infos.get(parentId); + IgfsEntryInfo parentInfo = infos.get(parentId); assert parentInfo != null; @@ -1448,7 +1457,7 @@ public class IgfsMetaManager extends IgfsManager { for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) { IgniteUuid trashId = IgfsUtils.trashId(i); - IgfsFileInfo trashInfo = getInfo(trashId); + IgfsEntryInfo trashInfo = getInfo(trashId); if (trashInfo != null && trashInfo.hasChildren()) { for (IgfsListingEntry entry : trashInfo.listing().values()) @@ -1474,7 +1483,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(final IgniteUuid fileId, Map props) + @Nullable private IgfsEntryInfo updatePropertiesNonTx(final IgniteUuid fileId, Map props) throws IgniteCheckedException { assert fileId != null; assert !F.isEmpty(props) : "Expects not-empty file's properties"; @@ -1485,7 +1494,7 @@ public class IgfsMetaManager extends IgfsManager { log.debug("Update file properties [fileId=" + fileId + ", props=" + props + ']'); try { - final IgfsFileInfo oldInfo = info(fileId); + final IgfsEntryInfo oldInfo = info(fileId); if (oldInfo == null) return null; @@ -1505,7 +1514,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 public IgfsFileInfo updateProperties(IgniteUuid fileId, Map props) + @Nullable public IgfsEntryInfo updateProperties(IgniteUuid fileId, Map props) throws IgniteCheckedException { if (busyLock.enterBusy()) { try { @@ -1514,7 +1523,7 @@ public class IgfsMetaManager extends IgfsManager { IgniteInternalTx tx = startTx(); try { - IgfsFileInfo info = updatePropertiesNonTx(fileId, props); + IgfsEntryInfo info = updatePropertiesNonTx(fileId, props); tx.commit(); @@ -1542,7 +1551,7 @@ public class IgfsMetaManager extends IgfsManager { * @param affRange Affinity range. * @return New file info. */ - public IgfsFileInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange) + public IgfsEntryInfo reserveSpace(IgfsPath path, IgniteUuid fileId, long space, IgfsFileAffinityRange affRange) throws IgniteCheckedException { validTxState(false); @@ -1555,12 +1564,12 @@ public class IgfsMetaManager extends IgfsManager { try { // Lock file ID for this transaction. - IgfsFileInfo oldInfo = info(fileId); + IgfsEntryInfo oldInfo = info(fileId); if (oldInfo == null) throw fsException("File has been deleted concurrently [path=" + path + ", id=" + fileId + ']'); - IgfsFileInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange)); + IgfsEntryInfo newInfo = invokeAndGet(fileId, new FileReserveSpaceProcessor(space, affRange)); tx.commit(); @@ -1590,8 +1599,8 @@ public class IgfsMetaManager extends IgfsManager { * @return Updated file info or {@code null} if such file ID not found. * @throws IgniteCheckedException If failed. */ - @Nullable public IgfsFileInfo updateInfo(IgniteUuid fileId, - EntryProcessor proc) throws IgniteCheckedException { + @Nullable public IgfsEntryInfo updateInfo(IgniteUuid fileId, + EntryProcessor proc) throws IgniteCheckedException { validTxState(false); assert fileId != null; assert proc != null; @@ -1605,12 +1614,12 @@ public class IgfsMetaManager extends IgfsManager { try { // Lock file ID for this transaction. - IgfsFileInfo oldInfo = info(fileId); + IgfsEntryInfo oldInfo = info(fileId); if (oldInfo == null) return null; // File not found. - IgfsFileInfo newInfo = invokeAndGet(fileId, proc); + IgfsEntryInfo newInfo = invokeAndGet(fileId, proc); if (newInfo == null) throw fsException("Failed to update file info with null value" + @@ -1673,7 +1682,7 @@ public class IgfsMetaManager extends IgfsManager { IgniteInternalTx tx = startTx(); try { - final Map lockInfos = lockIds(lockIds); + final Map lockInfos = lockIds(lockIds); if (!pathIds.verifyIntegrity(lockInfos)) // Directory structure changed concurrently. So we simply re-try. @@ -1779,7 +1788,7 @@ public class IgfsMetaManager extends IgfsManager { * @param name Name in parent. * @throws IgniteCheckedException If failed. */ - private void createNewEntry(IgfsFileInfo info, IgniteUuid parentId, String name) throws IgniteCheckedException { + private void createNewEntry(IgfsEntryInfo info, IgniteUuid parentId, String name) throws IgniteCheckedException { validTxState(true); if (!id2InfoPrj.putIfAbsent(info.id(), info)) @@ -1815,7 +1824,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Resulting file info. * @throws IgniteCheckedException If failed. */ - private IgfsFileInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException { + private IgfsEntryInfo invokeLock(IgniteUuid id, boolean delete) throws IgniteCheckedException { return invokeAndGet(id, new FileLockProcessor(createFileLockId(delete))); } @@ -1840,7 +1849,7 @@ public class IgfsMetaManager extends IgfsManager { * @return New file info. * @throws IgniteCheckedException If failed. */ - private IgfsFileInfo invokeAndGet(IgniteUuid id, EntryProcessor proc) + private IgfsEntryInfo invokeAndGet(IgniteUuid id, EntryProcessor proc) throws IgniteCheckedException { validTxState(true); @@ -1854,7 +1863,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Info. * @throws IgniteCheckedException If failed. */ - @Nullable private IgfsFileInfo getInfo(IgniteUuid id) throws IgniteCheckedException { + @Nullable private IgfsEntryInfo getInfo(IgniteUuid id) throws IgniteCheckedException { return id2InfoPrj.get(id); } @@ -1865,7 +1874,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Infos map. * @throws IgniteCheckedException If failed. */ - private Map getInfos(Collection ids) throws IgniteCheckedException { + private Map getInfos(Collection ids) throws IgniteCheckedException { return id2InfoPrj.getAll(ids); } @@ -1909,7 +1918,7 @@ public class IgfsMetaManager extends IgfsManager { private OutputStream out; @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map infos) throws Exception { + IgfsEntryInfo> infos) throws Exception { validTxState(true); assert !infos.isEmpty(); @@ -1924,7 +1933,7 @@ public class IgfsMetaManager extends IgfsManager { assert parentPath != null; - IgfsFileInfo parentInfo = infos.get(parentPath); + IgfsEntryInfo parentInfo = infos.get(parentPath); // Delegate to the secondary file system. out = simpleCreate ? fs.create(path, overwrite) : @@ -1963,15 +1972,23 @@ public class IgfsMetaManager extends IgfsManager { throw fsException("Failed to open output stream to the file created in " + "the secondary file system because the path points to a directory: " + path); - IgfsFileInfo newInfo = new IgfsFileInfo(status.blockSize(), status.length(), affKey, - createFileLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties(), - status.accessTime(), status.modificationTime()); + IgfsEntryInfo newInfo = IgfsUtils.createFile( + IgniteUuid.randomUuid(), + status.blockSize(), + status.length(), + affKey, + createFileLockId(false), + igfsCtx.igfs().evictExclude(path, false), + status.properties(), + status.accessTime(), + status.modificationTime() + ); // Add new file info to the listing optionally removing the previous one. IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), path.name(), newInfo); if (oldId != null) { - IgfsFileInfo oldInfo = info(oldId); + IgfsEntryInfo oldInfo = info(oldId); assert oldInfo != null; // Otherwise cache is in inconsistent state. @@ -2047,10 +2064,10 @@ public class IgfsMetaManager extends IgfsManager { private OutputStream out; @Override public IgfsSecondaryOutputStreamDescriptor onSuccess(Map infos) throws Exception { + IgfsEntryInfo> infos) throws Exception { validTxState(true); - final IgfsFileInfo info = infos.get(path); + final IgfsEntryInfo info = infos.get(path); if (info.isDirectory()) throw fsException("Failed to open output stream to the file in the " + @@ -2083,7 +2100,7 @@ public class IgfsMetaManager extends IgfsManager { } // Set lock and return. - IgfsFileInfo lockedInfo = invokeLock(info.id(), false); + IgfsEntryInfo lockedInfo = invokeLock(info.id(), false); return new IgfsSecondaryOutputStreamDescriptor(infos.get(path.parent()).id(), lockedInfo, out); @@ -2129,7 +2146,7 @@ public class IgfsMetaManager extends IgfsManager { assert path != null; // First, try getting file info without any transactions and synchronization. - IgfsFileInfo info = info(fileId(path)); + IgfsEntryInfo info = info(fileId(path)); if (info != null) { if (!info.isFile()) @@ -2143,8 +2160,8 @@ public class IgfsMetaManager extends IgfsManager { SynchronizationTask task = new SynchronizationTask() { @Override public IgfsSecondaryInputStreamDescriptor onSuccess( - Map infos) throws Exception { - IgfsFileInfo info = infos.get(path); + Map infos) throws Exception { + IgfsEntryInfo info = infos.get(path); if (info == null) throw fsException(new IgfsPathNotFoundException("File not found: " + path)); @@ -2183,7 +2200,7 @@ public class IgfsMetaManager extends IgfsManager { * @return File info or {@code null} if file not found. * @throws IgniteCheckedException If sync task failed. */ - @Nullable public IgfsFileInfo synchronizeFileDual(final IgfsSecondaryFileSystem fs, final IgfsPath path) + @Nullable public IgfsEntryInfo synchronizeFileDual(final IgfsSecondaryFileSystem fs, final IgfsPath path) throws IgniteCheckedException { assert fs != null; assert path != null; @@ -2191,20 +2208,20 @@ public class IgfsMetaManager extends IgfsManager { if (busyLock.enterBusy()) { try { // First, try getting file info without any transactions and synchronization. - IgfsFileInfo info = info(fileId(path)); + IgfsEntryInfo info = info(fileId(path)); if (info != null) return info; // If failed, try synchronize. - SynchronizationTask task = - new SynchronizationTask() { - @Override public IgfsFileInfo onSuccess(Map infos) + SynchronizationTask task = + new SynchronizationTask() { + @Override public IgfsEntryInfo onSuccess(Map infos) throws Exception { return infos.get(path); } - @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException { + @Override public IgfsEntryInfo onFailure(@Nullable Exception err) throws IgniteCheckedException { throw new IgniteCheckedException("Failed to synchronize path due to secondary file " + "system exception: " + path, err); } @@ -2246,7 +2263,7 @@ public class IgfsMetaManager extends IgfsManager { final Deque pendingEvts = new LinkedList<>(); SynchronizationTask task = new SynchronizationTask() { - @Override public Boolean onSuccess(Map infos) throws Exception { + @Override public Boolean onSuccess(Map infos) throws Exception { fs.mkdirs(path, props); assert !infos.isEmpty(); @@ -2261,7 +2278,7 @@ public class IgfsMetaManager extends IgfsManager { assert parentPath != null; - IgfsFileInfo parentPathInfo = infos.get(parentPath); + IgfsEntryInfo parentPathInfo = infos.get(parentPath); synchronize(fs, parentPath, parentPathInfo, path, true, null); @@ -2330,11 +2347,11 @@ public class IgfsMetaManager extends IgfsManager { final Collection pendingEvts = new LinkedList<>(); SynchronizationTask task = new SynchronizationTask() { - @Override public Boolean onSuccess(Map infos) throws Exception { - IgfsFileInfo srcInfo = infos.get(src); - IgfsFileInfo srcParentInfo = infos.get(src.parent()); - IgfsFileInfo destInfo = infos.get(dest); - IgfsFileInfo destParentInfo = dest.parent() != null ? infos.get(dest.parent()) : null; + @Override public Boolean onSuccess(Map infos) throws Exception { + IgfsEntryInfo srcInfo = infos.get(src); + IgfsEntryInfo srcParentInfo = infos.get(src.parent()); + IgfsEntryInfo destInfo = infos.get(dest); + IgfsEntryInfo destParentInfo = dest.parent() != null ? infos.get(dest.parent()) : null; // Source path and destination (or destination parent) must exist. if (srcInfo == null) @@ -2424,8 +2441,8 @@ public class IgfsMetaManager extends IgfsManager { final IgniteUuid trashId = IgfsUtils.randomTrashId(); SynchronizationTask task = new SynchronizationTask() { - @Override public Boolean onSuccess(Map infos) throws Exception { - IgfsFileInfo info = infos.get(path); + @Override public Boolean onSuccess(Map infos) throws Exception { + IgfsEntryInfo info = infos.get(path); if (info == null) return false; // File doesn't exist in the secondary file system. @@ -2482,7 +2499,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Update file info. * @throws IgniteCheckedException If update failed. */ - public IgfsFileInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, + public IgfsEntryInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final Map props) throws IgniteCheckedException { assert fs != null; assert path != null; @@ -2490,8 +2507,8 @@ public class IgfsMetaManager extends IgfsManager { if (busyLock.enterBusy()) { try { - SynchronizationTask task = new SynchronizationTask() { - @Override public IgfsFileInfo onSuccess(Map infos) throws Exception { + SynchronizationTask task = new SynchronizationTask() { + @Override public IgfsEntryInfo onSuccess(Map infos) throws Exception { if (infos.get(path) == null) return null; @@ -2500,7 +2517,7 @@ public class IgfsMetaManager extends IgfsManager { return updatePropertiesNonTx(infos.get(path).id(), props); } - @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException { + @Override public IgfsEntryInfo onFailure(@Nullable Exception err) throws IgniteCheckedException { U.error(log, "Path update in DUAL mode failed [path=" + path + ", properties=" + props + ']', err); @@ -2531,12 +2548,12 @@ public class IgfsMetaManager extends IgfsManager { * @return File info of the end path. * @throws IgniteCheckedException If failed. */ - private IgfsFileInfo synchronize(IgfsSecondaryFileSystem fs, + private IgfsEntryInfo synchronize(IgfsSecondaryFileSystem fs, IgfsPath startPath, - IgfsFileInfo startPathInfo, + IgfsEntryInfo startPathInfo, IgfsPath endPath, boolean strict, - @Nullable Map created) + @Nullable Map created) throws IgniteCheckedException { assert fs != null; @@ -2544,7 +2561,7 @@ public class IgfsMetaManager extends IgfsManager { validTxState(true); - IgfsFileInfo parentInfo = startPathInfo; + IgfsEntryInfo parentInfo = startPathInfo; List components = endPath.components(); @@ -2585,11 +2602,25 @@ public class IgfsMetaManager extends IgfsManager { } // Recreate the path locally. - IgfsFileInfo curInfo = status.isDirectory() ? - new IgfsFileInfo(true, status.properties(), status.accessTime(), status.modificationTime()) : - new IgfsFileInfo(igfsCtx.configuration().getBlockSize(), status.length(), - igfsCtx.igfs().evictExclude(curPath, false), status.properties(), - status.accessTime(), status.modificationTime()); + IgfsEntryInfo curInfo = status.isDirectory() ? + IgfsUtils.createDirectory( + IgniteUuid.randomUuid(), + null, + status.properties(), + status.accessTime(), + status.modificationTime() + ) : + IgfsUtils.createFile( + IgniteUuid.randomUuid(), + igfsCtx.configuration().getBlockSize(), + status.length(), + null, + null, + igfsCtx.igfs().evictExclude(curPath, false), + status.properties(), + status.accessTime(), + status.modificationTime() + ); IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), components.get(i), curInfo); @@ -2719,7 +2750,7 @@ public class IgfsMetaManager extends IgfsManager { lockArr[idx++] = id; } - Map idToInfo = lockIds(lockArr); + Map idToInfo = lockIds(lockArr); if (extraLockIds != null) { for (IgniteUuid id : extraLockIds) @@ -2762,9 +2793,9 @@ public class IgfsMetaManager extends IgfsManager { continue; // Release all locks and try again. else { // Perform synchronization. - Map infos = new HashMap<>(); + Map infos = new HashMap<>(); - TreeMap created = new TreeMap<>(); + TreeMap created = new TreeMap<>(); for (IgfsPath path : paths) { IgfsPath parentPath = path.parent(); @@ -2781,7 +2812,7 @@ public class IgfsMetaManager extends IgfsManager { assert firstParentPath != null; assert pathToId.get(firstParentPath) != null; - IgfsFileInfo info = synchronize(fs, + IgfsEntryInfo info = synchronize(fs, firstParentPath, idToInfo.get(pathToId.get(firstParentPath)), path, @@ -2899,15 +2930,15 @@ public class IgfsMetaManager extends IgfsManager { IgniteInternalTx tx = startTx(); try { - Map infoMap = lockIds(fileId, parentId); + Map infoMap = lockIds(fileId, parentId); - IgfsFileInfo fileInfo = infoMap.get(fileId); + IgfsEntryInfo fileInfo = infoMap.get(fileId); if (fileInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to update times " + "(path was not found): " + fileName)); - IgfsFileInfo parentInfo = infoMap.get(parentId); + IgfsEntryInfo parentInfo = infoMap.get(parentId); if (parentInfo == null) throw fsException(new IgfsPathNotFoundException("Failed to update times " + @@ -2967,7 +2998,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Task result. * @throws Exception If failed. */ - public T onSuccess(Map infos) throws Exception; + public T onSuccess(Map infos) throws Exception; /** * Callback handler in case synchronization failed. @@ -2993,7 +3024,7 @@ public class IgfsMetaManager extends IgfsManager { private IgfsPath parentPath; /** Parent path info. */ - private IgfsFileInfo parentInfo; + private IgfsEntryInfo parentInfo; /** * Constructor. @@ -3003,7 +3034,7 @@ public class IgfsMetaManager extends IgfsManager { * @param parentPath Parent path. * @param parentInfo Parent info. */ - PathDescriptor(IgfsPath path, List ids, IgfsPath parentPath, IgfsFileInfo parentInfo) { + PathDescriptor(IgfsPath path, List ids, IgfsPath parentPath, IgfsEntryInfo parentInfo) { assert path != null; assert ids != null && !ids.isEmpty(); assert parentPath == null && parentInfo == null || parentPath != null && parentInfo != null; @@ -3040,7 +3071,7 @@ public class IgfsMetaManager extends IgfsManager { * @param newParentPath New parent path. * @param newParentInfo New parent info. */ - private void updateParent(IgfsPath newParentPath, IgfsFileInfo newParentInfo) { + private void updateParent(IgfsPath newParentPath, IgfsEntryInfo newParentInfo) { assert newParentPath != null; assert newParentInfo != null; assert path.isSubDirectoryOf(newParentPath); @@ -3065,7 +3096,7 @@ public class IgfsMetaManager extends IgfsManager { * * @return Parent path info. */ - private IgfsFileInfo parentInfo() { + private IgfsEntryInfo parentInfo() { return parentInfo; } } @@ -3074,7 +3105,7 @@ public class IgfsMetaManager extends IgfsManager { * Remove entry from directory listing. */ @GridInternal - private static final class ListingRemoveProcessor implements EntryProcessor, + private static final class ListingRemoveProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3104,9 +3135,9 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry e, Object... args) + @Override public Void process(MutableEntry e, Object... args) throws EntryProcessorException { - IgfsFileInfo fileInfo = e.getValue(); + IgfsEntryInfo fileInfo = e.getValue(); assert fileInfo != null; assert fileInfo.isDirectory(); @@ -3124,7 +3155,7 @@ public class IgfsMetaManager extends IgfsManager { // Modify listing in-place. listing.remove(fileName); - e.setValue(new IgfsFileInfo(listing, fileInfo)); + e.setValue(fileInfo.listing(listing)); return null; } @@ -3146,7 +3177,7 @@ public class IgfsMetaManager extends IgfsManager { * Update directory listing closure. */ @GridInternal - private static final class ListingAddProcessor implements EntryProcessor, + private static final class ListingAddProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3180,8 +3211,8 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry e, Object... args) { - IgfsFileInfo fileInfo = e.getValue(); + @Override public Void process(MutableEntry e, Object... args) { + IgfsEntryInfo fileInfo = e.getValue(); assert fileInfo.isDirectory(); @@ -3195,7 +3226,7 @@ public class IgfsMetaManager extends IgfsManager { " [listing=" + listing + ", fileName=" + fileName + ", entry=" + entry + ", oldEntry=" + oldEntry + ']'); - e.setValue(new IgfsFileInfo(listing, fileInfo)); + e.setValue(fileInfo.listing(listing)); return null; } @@ -3221,7 +3252,7 @@ public class IgfsMetaManager extends IgfsManager { /** * Listing replace processor. */ - private static final class ListingReplaceProcessor implements EntryProcessor, + private static final class ListingReplaceProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3251,9 +3282,9 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry e, Object... args) + @Override public Void process(MutableEntry e, Object... args) throws EntryProcessorException { - IgfsFileInfo fileInfo = e.getValue(); + IgfsEntryInfo fileInfo = e.getValue(); assert fileInfo.isDirectory(); @@ -3267,7 +3298,7 @@ public class IgfsMetaManager extends IgfsManager { listing.put(name, new IgfsListingEntry(id, oldEntry.isDirectory())); - e.setValue(new IgfsFileInfo(listing, fileInfo)); + e.setValue(fileInfo.listing(listing)); return null; } @@ -3289,7 +3320,7 @@ public class IgfsMetaManager extends IgfsManager { * Update path closure. */ @GridInternal - private static final class UpdatePathProcessor implements EntryProcessor, + private static final class UpdatePathProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3312,10 +3343,12 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry e, Object... args) { - IgfsFileInfo info = e.getValue(); + @Override public Void process(MutableEntry e, Object... args) { + IgfsEntryInfo info = e.getValue(); - e.setValue(builder(info).path(path).build()); + IgfsEntryInfo newInfo = info.path(path); + + e.setValue(newInfo); return null; } @@ -3349,7 +3382,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Tuple containing the file info and its parent id. * @throws IgniteCheckedException If failed. */ - IgniteBiTuple append( + IgniteBiTuple append( final IgfsPath path, Map dirProps, final boolean create, @@ -3379,7 +3412,7 @@ public class IgfsMetaManager extends IgfsManager { IgniteInternalTx tx = startTx(); try { - Map lockInfos = lockIds(lockIds); + Map lockInfos = lockIds(lockIds); if (!pathIds.verifyIntegrity(lockInfos)) // Directory structure changed concurrently. So we simply re-try. @@ -3387,7 +3420,7 @@ public class IgfsMetaManager extends IgfsManager { if (pathIds.allExists()) { // All participants are found. Simply open the stream. - IgfsFileInfo info = lockInfos.get(pathIds.lastId()); + IgfsEntryInfo info = lockInfos.get(pathIds.lastId()); // Check: is it a file? if (!info.isFile()) @@ -3400,7 +3433,7 @@ public class IgfsMetaManager extends IgfsManager { // At this point we can open the stream safely. info = invokeLock(info.id(), false); - IgniteBiTuple t2 = new T2<>(info, pathIds.lastParentId()); + IgniteBiTuple t2 = new T2<>(info, pathIds.lastParentId()); tx.commit(); @@ -3450,7 +3483,7 @@ public class IgfsMetaManager extends IgfsManager { * @param simpleCreate Whether new file should be created in secondary FS using create(Path, boolean) method. * @return Tuple containing the created file info and its parent id. */ - IgniteBiTuple create( + IgniteBiTuple create( final IgfsPath path, Map dirProps, final boolean overwrite, @@ -3488,7 +3521,7 @@ public class IgfsMetaManager extends IgfsManager { IgniteInternalTx tx = startTx(); try { - Map lockInfos = lockIds(lockIds); + Map lockInfos = lockIds(lockIds); if (!pathIds.verifyIntegrity(lockInfos)) // Directory structure changed concurrently. So we simply re-try. @@ -3496,7 +3529,7 @@ public class IgfsMetaManager extends IgfsManager { if (pathIds.allExists()) { // All participants found. - IgfsFileInfo oldInfo = lockInfos.get(pathIds.lastId()); + IgfsEntryInfo oldInfo = lockInfos.get(pathIds.lastId()); // Check: is it a file? if (!oldInfo.isFile()) @@ -3527,14 +3560,14 @@ public class IgfsMetaManager extends IgfsManager { // Third step: create the file. long createTime = System.currentTimeMillis(); - IgfsFileInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime, + IgfsEntryInfo newInfo = invokeAndGet(overwriteId, new FileCreateProcessor(createTime, fileProps, blockSize, affKey, createFileLockId(false), evictExclude)); // Fourth step: update path of remove file. invokeUpdatePath(oldId, path); // Prepare result and commit. - IgniteBiTuple t2 = new T2<>(newInfo, parentId); + IgniteBiTuple t2 = new T2<>(newInfo, parentId); tx.commit(); @@ -3581,7 +3614,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Result or {@code} if the first parent already contained child with the same name. * @throws IgniteCheckedException If failed. */ - @Nullable IgfsPathsCreateResult createDirectory(IgfsPathIds pathIds, Map lockInfos, + @Nullable IgfsPathsCreateResult createDirectory(IgfsPathIds pathIds, Map lockInfos, Map dirProps) throws IgniteCheckedException { // Check if entry we are going to write to is directory. if (lockInfos.get(pathIds.lastExistingId()).isFile()) @@ -3604,7 +3637,7 @@ public class IgfsMetaManager extends IgfsManager { * @return Result or {@code} if the first parent already contained child with the same name. * @throws IgniteCheckedException If failed. */ - @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map lockInfos, + @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map lockInfos, Map dirProps, Map fileProps, int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException{ // Check if entry we are going to write to is directory. @@ -3630,11 +3663,11 @@ public class IgfsMetaManager extends IgfsManager { * @throws IgniteCheckedException If failed. */ private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds, - Map lockInfos, Map dirProps, Map fileProps, + Map lockInfos, Map dirProps, Map fileProps, int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException { // This is our starting point. int lastExistingIdx = pathIds.lastExistingIndex(); - IgfsFileInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId()); + IgfsEntryInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId()); // If current info already contains entry with the same name as it's child, then something // has changed concurrently. We must re-try because we cannot get info of this unexpected @@ -3684,7 +3717,7 @@ public class IgfsMetaManager extends IgfsManager { } // Third step: create leaf. - IgfsFileInfo info; + IgfsEntryInfo info; if (dir) info = invokeAndGet(curId, new DirectoryCreateProcessor(createTime, dirProps)); @@ -3723,7 +3756,7 @@ public class IgfsMetaManager extends IgfsManager { /** * File create processor. */ - private static class FileCreateProcessor implements EntryProcessor, + private static class FileCreateProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3774,12 +3807,19 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public IgfsFileInfo process(MutableEntry entry, Object... args) + @Override public IgfsEntryInfo process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, affKey, lockId, evictExclude, props, - createTime, createTime); - - info.id(entry.getKey()); + IgfsEntryInfo info = IgfsUtils.createFile( + entry.getKey(), + blockSize, + 0L, + affKey, + lockId, + evictExclude, + props, + createTime, + createTime + ); entry.setValue(info); @@ -3810,7 +3850,7 @@ public class IgfsMetaManager extends IgfsManager { /** * Directory create processor. */ - private static class DirectoryCreateProcessor implements EntryProcessor, + private static class DirectoryCreateProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3861,15 +3901,19 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public IgfsFileInfo process(MutableEntry entry, Object... args) + @Override public IgfsEntryInfo process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo info = new IgfsFileInfo(true, props, createTime, createTime); + IgfsEntryInfo info = IgfsUtils.createDirectory( + entry.getKey(), + null, + props, + createTime, + createTime + ); if (childName != null) - info = new IgfsFileInfo(Collections.singletonMap(childName, childEntry), info); - - info.id(entry.getKey()); + info = info.listing(Collections.singletonMap(childName, childEntry)); entry.setValue(info); @@ -3906,7 +3950,7 @@ public class IgfsMetaManager extends IgfsManager { /** * File lock entry processor. */ - private static class FileLockProcessor implements EntryProcessor, + private static class FileLockProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3931,11 +3975,11 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public IgfsFileInfo process(MutableEntry entry, Object... args) + @Override public IgfsEntryInfo process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo oldInfo = entry.getValue(); + IgfsEntryInfo oldInfo = entry.getValue(); - IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, lockId, oldInfo.modificationTime()); + IgfsEntryInfo newInfo = oldInfo.lock(lockId); entry.setValue(newInfo); @@ -3956,7 +4000,7 @@ public class IgfsMetaManager extends IgfsManager { /** * File unlock entry processor. */ - private static class FileUnlockProcessor implements EntryProcessor, + private static class FileUnlockProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -3981,11 +4025,11 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry entry, Object... args) + @Override public Void process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo old = entry.getValue(); + IgfsEntryInfo old = entry.getValue(); - entry.setValue(new IgfsFileInfo(old, null, modificationTime)); + entry.setValue(old.unlock(modificationTime)); return null; } @@ -4004,7 +4048,7 @@ public class IgfsMetaManager extends IgfsManager { /** * File reserve space entry processor. */ - private static class FileReserveSpaceProcessor implements EntryProcessor, + private static class FileReserveSpaceProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -4034,17 +4078,15 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public IgfsFileInfo process(MutableEntry entry, Object... args) + @Override public IgfsEntryInfo process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo oldInfo = entry.getValue(); + IgfsEntryInfo oldInfo = entry.getValue(); IgfsFileMap newMap = new IgfsFileMap(oldInfo.fileMap()); newMap.addRange(affRange); - IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, oldInfo.length() + space); - - newInfo.fileMap(newMap); + IgfsEntryInfo newInfo = oldInfo.length(oldInfo.length() + space).fileMap(newMap); entry.setValue(newInfo); @@ -4067,7 +4109,7 @@ public class IgfsMetaManager extends IgfsManager { /** * Update properties processor. */ - private static class UpdatePropertiesProcessor implements EntryProcessor, + private static class UpdatePropertiesProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -4092,9 +4134,9 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public IgfsFileInfo process(MutableEntry entry, Object... args) + @Override public IgfsEntryInfo process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo oldInfo = entry.getValue(); + IgfsEntryInfo oldInfo = entry.getValue(); Map tmp = oldInfo.properties(); @@ -4109,7 +4151,7 @@ public class IgfsMetaManager extends IgfsManager { tmp.put(e.getKey(), e.getValue()); } - IgfsFileInfo newInfo = new IgfsFileInfo(oldInfo, tmp); + IgfsEntryInfo newInfo = oldInfo.properties(tmp); entry.setValue(newInfo); @@ -4130,7 +4172,7 @@ public class IgfsMetaManager extends IgfsManager { /** * Update times entry processor. */ - private static class UpdateTimesProcessor implements EntryProcessor, + private static class UpdateTimesProcessor implements EntryProcessor, Externalizable { /** */ private static final long serialVersionUID = 0L; @@ -4160,12 +4202,12 @@ public class IgfsMetaManager extends IgfsManager { } /** {@inheritDoc} */ - @Override public Void process(MutableEntry entry, Object... args) + @Override public Void process(MutableEntry entry, Object... args) throws EntryProcessorException { - IgfsFileInfo oldInfo = entry.getValue(); + IgfsEntryInfo oldInfo = entry.getValue(); - entry.setValue(new IgfsFileInfo(oldInfo, accessTime, modificationTime)); + entry.setValue(oldInfo.accessModificationTime(accessTime, modificationTime)); return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java index 3bf1011..ef2826b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java @@ -17,10 +17,6 @@ package org.apache.ignite.internal.processors.igfs; -import java.io.DataInput; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.igfs.IgfsException; import org.apache.ignite.igfs.IgfsMode; @@ -33,6 +29,11 @@ import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; +import java.io.DataInput; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; + import static org.apache.ignite.igfs.IgfsMode.DUAL_SYNC; import static org.apache.ignite.igfs.IgfsMode.PRIMARY; import static org.apache.ignite.igfs.IgfsMode.PROXY; @@ -55,7 +56,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter { /** File descriptor. */ @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized") - private IgfsFileInfo fileInfo; + private IgfsEntryInfo fileInfo; /** Parent ID. */ private final IgniteUuid parentId; @@ -102,7 +103,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter { * @param batch Optional secondary file system batch. * @param metrics Local IGFS metrics. */ - IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo, IgniteUuid parentId, + IgfsOutputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsEntryInfo fileInfo, IgniteUuid parentId, int bufSize, IgfsMode mode, @Nullable IgfsFileWorkerBatch batch, IgfsLocalMetrics metrics) { super(path, optimizeBufferSize(bufSize, fileInfo)); @@ -143,7 +144,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter { * @return Optimized buffer size. */ @SuppressWarnings("IfMayBeConditional") - private static int optimizeBufferSize(int bufSize, IgfsFileInfo fileInfo) { + private static int optimizeBufferSize(int bufSize, IgfsEntryInfo fileInfo) { assert bufSize > 0; if (fileInfo == null) @@ -292,7 +293,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter { if (space > 0) { data.awaitAllAcksReceived(fileInfo.id()); - IgfsFileInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange); + IgfsEntryInfo fileInfo0 = meta.reserveSpace(path, fileInfo.id(), space, streamRange); if (fileInfo0 == null) throw new IOException("File was concurrently deleted: " + path); @@ -407,7 +408,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter { * @param fileInfo File info to build initial range for. * @return Affinity range. */ - private IgfsFileAffinityRange initialStreamRange(IgfsFileInfo fileInfo) { + private IgfsFileAffinityRange initialStreamRange(IgfsEntryInfo fileInfo) { if (!igfsCtx.configuration().isFragmentizerEnabled()) return null; http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java index 1f669b0..2903239 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathIds.java @@ -267,10 +267,10 @@ public class IgfsPathIds { * @param infos Info. * @return {@code True} if full integrity is preserved. */ - public boolean verifyIntegrity(Map infos) { + public boolean verifyIntegrity(Map infos) { for (int i = 0; i <= lastExistingIdx; i++) { IgniteUuid curId = ids[i]; - IgfsFileInfo curInfo = infos.get(curId); + IgfsEntryInfo curInfo = infos.get(curId); // Check if required ID is there. if (curInfo == null) http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java index 3b620f8..9462aa4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java @@ -31,7 +31,7 @@ public class IgfsPathsCreateResult { private final List paths; /** Info of the last created file. */ - private final IgfsFileInfo info; + private final IgfsEntryInfo info; /** Parent ID. */ private final IgniteUuid parentId; @@ -43,7 +43,7 @@ public class IgfsPathsCreateResult { * @param info Info of the last created file. * @param parentId Parent ID. */ - public IgfsPathsCreateResult(List paths, IgfsFileInfo info, IgniteUuid parentId) { + public IgfsPathsCreateResult(List paths, IgfsEntryInfo info, IgniteUuid parentId) { this.paths = paths; this.info = info; this.parentId = parentId; @@ -59,7 +59,7 @@ public class IgfsPathsCreateResult { /** * @return Info of the last created file. */ - public IgfsFileInfo info() { + public IgfsEntryInfo info() { return info; } http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java index 54c0525..76d6be8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java @@ -24,7 +24,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl */ public class IgfsSecondaryInputStreamDescriptor { /** File info in the primary file system. */ - private final IgfsFileInfo info; + private final IgfsEntryInfo info; /** Secondary file system input stream wrapper. */ private final IgfsSecondaryFileSystemPositionedReadable secReader; @@ -35,7 +35,7 @@ public class IgfsSecondaryInputStreamDescriptor { * @param info File info in the primary file system. * @param secReader Secondary file system reader. */ - IgfsSecondaryInputStreamDescriptor(IgfsFileInfo info, IgfsSecondaryFileSystemPositionedReadable secReader) { + IgfsSecondaryInputStreamDescriptor(IgfsEntryInfo info, IgfsSecondaryFileSystemPositionedReadable secReader) { assert info != null; assert secReader != null; @@ -46,7 +46,7 @@ public class IgfsSecondaryInputStreamDescriptor { /** * @return File info in the primary file system. */ - IgfsFileInfo info() { + IgfsEntryInfo info() { return info; } http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java index dd6372f..507ccfc 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryOutputStreamDescriptor.java @@ -17,9 +17,10 @@ package org.apache.ignite.internal.processors.igfs; -import java.io.OutputStream; import org.apache.ignite.lang.IgniteUuid; +import java.io.OutputStream; + /** * Descriptor of an output stream opened to the secondary file system. */ @@ -28,7 +29,7 @@ public class IgfsSecondaryOutputStreamDescriptor { private final IgniteUuid parentId; /** File info in the primary file system. */ - private final IgfsFileInfo info; + private final IgfsEntryInfo info; /** Output stream to the secondary file system. */ private final OutputStream out; @@ -40,7 +41,7 @@ public class IgfsSecondaryOutputStreamDescriptor { * @param info File info in the primary file system. * @param out Output stream to the secondary file system. */ - IgfsSecondaryOutputStreamDescriptor(IgniteUuid parentId, IgfsFileInfo info, OutputStream out) { + IgfsSecondaryOutputStreamDescriptor(IgniteUuid parentId, IgfsEntryInfo info, OutputStream out) { assert parentId != null; assert info != null; assert out != null; @@ -60,7 +61,7 @@ public class IgfsSecondaryOutputStreamDescriptor { /** * @return File info in the primary file system. */ - IgfsFileInfo info() { + IgfsEntryInfo info() { return info; } http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java index ef7d5c7..325f636 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java @@ -43,6 +43,7 @@ import org.apache.ignite.transactions.Transaction; import org.jetbrains.annotations.Nullable; import java.lang.reflect.Constructor; +import java.util.Map; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; @@ -321,4 +322,71 @@ public class IgfsUtils { } } } + + /** + * Create empty directory with the given ID. + * + * @param id ID. + * @return File info. + */ + public static IgfsDirectoryInfo createDirectory(IgniteUuid id) { + return createDirectory(id, null, null); + } + + /** + * Create directory. + * + * @param id ID. + * @param listing Listing. + * @param props Properties. + * @return File info. + */ + public static IgfsDirectoryInfo createDirectory( + IgniteUuid id, + @Nullable Map listing, + @Nullable Map props) { + long time = System.currentTimeMillis(); + + return createDirectory(id, listing, props, time, time); + } + + /** + * Create directory. + * + * @param id ID. + * @param listing Listing. + * @param props Properties. + * @param createTime Create time. + * @param modificationTime Modification time. + * @return File info. + */ + public static IgfsDirectoryInfo createDirectory( + IgniteUuid id, + @Nullable Map listing, + @Nullable Map props, + long createTime, + long modificationTime) { + return new IgfsDirectoryInfo(id, listing, props, createTime, modificationTime); + } + + /** + * Create file. + * + * @param id File ID. + * @param blockSize Block size. + * @param len Length. + * @param affKey Affinity key. + * @param lockId Lock ID. + * @param evictExclude Evict exclude flag. + * @param props Properties. + * @param accessTime Access time. + * @param modificationTime Modification time. + * @return File info. + */ + public static IgfsFileInfo createFile(IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey, + @Nullable IgniteUuid lockId, boolean evictExclude, @Nullable Map props, long accessTime, + long modificationTime) { + return new IgfsFileInfo(id, blockSize, len, affKey, props, null, lockId, accessTime, modificationTime, + evictExclude); + } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java index ea70f49..20051ce 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/package-info.java @@ -17,6 +17,6 @@ /** * - * Contains high performance file system processer. + * Contains high performance file system processor. */ package org.apache.ignite.internal.processors.igfs; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java index 22f427a..d52a918 100644 --- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java @@ -22,8 +22,8 @@ import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.FileSystemConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest; +import org.apache.ignite.internal.processors.igfs.IgfsEntryInfo; import org.apache.ignite.internal.processors.igfs.IgfsEx; -import org.apache.ignite.internal.processors.igfs.IgfsFileInfo; import org.apache.ignite.internal.processors.igfs.IgfsMetaManager; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.lang.IgniteUuid; @@ -139,7 +139,7 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest { if (fileId == null) throw new IgfsPathNotFoundException("File not found: " + path); - IgfsFileInfo fileInfo = meta.info(fileId); + IgfsEntryInfo fileInfo = meta.info(fileId); do { if (fileInfo == null) http://git-wip-us.apache.org/repos/asf/ignite/blob/4794f87b/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 2acf59c..edec572 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 @@ -3081,7 +3081,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { * @param igfs The IGFS instance. * @return The data cache. */ - protected static GridCacheAdapter getMetaCache(IgniteFileSystem igfs) { + protected static GridCacheAdapter getMetaCache(IgniteFileSystem igfs) { String dataCacheName = igfs.configuration().getMetaCacheName(); IgniteEx igniteEx = ((IgfsEx)igfs).context().kernalContext().grid(); @@ -3111,7 +3111,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { entry.getValue().await(); } catch (IgniteCheckedException e) { - if (!entry.getValue().cancelled()) + if (!(e instanceof IgfsFileWorkerBatchCancelledException)) throw e; } }