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 F1ECA19903 for ; Mon, 14 Mar 2016 08:19:50 +0000 (UTC) Received: (qmail 25698 invoked by uid 500); 14 Mar 2016 08:19:50 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 25661 invoked by uid 500); 14 Mar 2016 08:19:50 -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 25652 invoked by uid 99); 14 Mar 2016 08:19:50 -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; Mon, 14 Mar 2016 08:19:50 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 85B96DFBC9; Mon, 14 Mar 2016 08:19:50 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Mon, 14 Mar 2016 08:19:51 -0000 Message-Id: In-Reply-To: <33e3daf1a30b4a32b772b4c845c47d29@git.apache.org> References: <33e3daf1a30b4a32b772b4c845c47d29@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/2] ignite git commit: IGNITE-2810: Feature implemented. IGNITE-2810: Feature implemented. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/421d0c1d Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/421d0c1d Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/421d0c1d Branch: refs/heads/ignite-2810 Commit: 421d0c1d5682e9dcd723a81f6caa6c85eb49ed17 Parents: 6c4c39c Author: vozerov-gridgain Authored: Mon Mar 14 11:19:45 2016 +0300 Committer: vozerov-gridgain Committed: Mon Mar 14 11:19:45 2016 +0300 ---------------------------------------------------------------------- .../processors/igfs/IgfsDeleteWorker.java | 25 +++-- .../processors/igfs/IgfsMetaManager.java | 103 ++++++++++--------- .../internal/processors/igfs/IgfsUtils.java | 68 +++++++++++- .../processors/igfs/IgfsAbstractSelfTest.java | 21 ++-- .../processors/igfs/IgfsSizeSelfTest.java | 9 +- 5 files changed, 154 insertions(+), 72 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/421d0c1d/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java index 65529d5..ffddd3e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDeleteWorker.java @@ -149,13 +149,23 @@ public class IgfsDeleteWorker extends IgfsThread { } /** - * Perform cleanup of the trash directory. + * Perform cleanup of trash directories. */ private void delete() { + for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) + delete(IgfsUtils.trashId(i)); + } + + /** + * Perform cleanup of concrete trash directory. + * + * @param trashId Trash ID. + */ + private void delete(IgniteUuid trashId) { IgfsFileInfo info = null; try { - info = meta.info(IgfsUtils.TRASH_ID); + info = meta.info(trashId); } catch(ClusterTopologyServerNotFoundException e) { LT.warn(log, e, "Server nodes not found."); @@ -173,7 +183,7 @@ public class IgfsDeleteWorker extends IgfsThread { try { if (!cancelled) { - if (delete(entry.getKey(), fileId)) { + if (delete(trashId, entry.getKey(), fileId)) { if (log.isDebugEnabled()) log.debug("Sending delete confirmation message [name=" + entry.getKey() + ", fileId=" + fileId + ']'); @@ -199,12 +209,13 @@ public class IgfsDeleteWorker extends IgfsThread { /** * Remove particular entry from the TRASH directory. * + * @param trashId ID of the trash directory. * @param name Entry name. * @param id Entry ID. * @return {@code True} in case the entry really was deleted form the file system by this call. * @throws IgniteCheckedException If failed. */ - private boolean delete(String name, IgniteUuid id) throws IgniteCheckedException { + private boolean delete(IgniteUuid trashId, String name, IgniteUuid id) throws IgniteCheckedException { assert name != null; assert id != null; @@ -213,10 +224,10 @@ public class IgfsDeleteWorker extends IgfsThread { if (info != null) { if (info.isDirectory()) { - if (!deleteDirectoryContents(IgfsUtils.TRASH_ID, id)) + if (!deleteDirectoryContents(trashId, id)) return false; - if (meta.delete(IgfsUtils.TRASH_ID, name, id)) + if (meta.delete(trashId, name, id)) return true; } else { @@ -234,7 +245,7 @@ public class IgfsDeleteWorker extends IgfsThread { // In case this node crashes, other node will re-delete the file. data.delete(lockedInfo).get(); - boolean ret = meta.delete(IgfsUtils.TRASH_ID, name, id); + boolean ret = meta.delete(trashId, name, id); if (info.path() != null) IgfsUtils.sendEvents(igfsCtx.kernalContext(), info.path(), EVT_IGFS_FILE_PURGED); http://git-wip-us.apache.org/repos/asf/ignite/blob/421d0c1d/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 c75a0eb..8c9544e 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 @@ -669,36 +669,37 @@ public class IgfsMetaManager extends IgfsManager { if (log.isDebugEnabled()) log.debug("Locked file ids: " + fileIds); - // Force root & trash IDs always exist in cache. - addInfoIfNeeded(fileIds, map, IgfsUtils.ROOT_ID); - addInfoIfNeeded(fileIds, map, IgfsUtils.TRASH_ID); + for (IgniteUuid fileId : fileIds) { + if (IgfsUtils.isRootOrTrashId(fileId)) { + if (!map.containsKey(fileId)) + map.put(fileId, createSystemEntryIfAbsent(fileId)); + } + } // Returns detail's map for locked IDs. return map; } /** - * Adds FileInfo into the cache if it is requested in fileIds and is not present in the map. + * create system entry if it is absent. * - * @param fileIds A list that may contain the id. - * @param map The map that may not contain the id. - * @param id The id to check. + * @param id System entry ID. * @throws IgniteCheckedException On error. */ - private void addInfoIfNeeded(Collection fileIds, Map map, IgniteUuid id) + private IgfsFileInfo createSystemEntryIfAbsent(IgniteUuid id) throws IgniteCheckedException { assert validTxState(true); - if (fileIds.contains(id) && !map.containsKey(id)) { - IgfsFileInfo info = new IgfsFileInfo(id); + assert IgfsUtils.isRootOrTrashId(id); - IgfsFileInfo anotherInfo = id2InfoPrj.getAndPutIfAbsent(id, info); + IgfsFileInfo info = new IgfsFileInfo(id); - if (anotherInfo != null) - info = anotherInfo; + IgfsFileInfo oldInfo = id2InfoPrj.getAndPutIfAbsent(id, info); - map.put(id, info); - } + if (oldInfo != null) + info = oldInfo; + + return info; } /** @@ -1106,11 +1107,13 @@ public class IgfsMetaManager extends IgfsManager { try { assert validTxState(false); + IgniteUuid trashId = IgfsUtils.randomTrashId(); + final IgniteInternalTx tx = startTx(); try { // NB: We may lock root because its id is less than any other id: - final IgfsFileInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, IgfsUtils.TRASH_ID).get(IgfsUtils.ROOT_ID); + final IgfsFileInfo rootInfo = lockIds(IgfsUtils.ROOT_ID, trashId).get(IgfsUtils.ROOT_ID); assert rootInfo != null; @@ -1129,7 +1132,7 @@ public class IgfsMetaManager extends IgfsManager { id2InfoPrj.put(newInfo.id(), newInfo); // Add new info to trash listing. - id2InfoPrj.invoke(IgfsUtils.TRASH_ID, new ListingAdd(newInfo.id().toString(), + id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(), new IgfsListingEntry(newInfo))); // Remove listing entries from root. @@ -1176,8 +1179,7 @@ public class IgfsMetaManager extends IgfsManager { final IgniteUuid victimId = pathIdList.get(pathIdList.size() - 1); - assert !IgfsUtils.TRASH_ID.equals(victimId) : - "TRASH does not have path, it cannot ever be deletion victim."; + assert !IgfsUtils.isTrashId(victimId) : "TRASH does not have path, it cannot ever be deletion victim."; assert !IgfsUtils.ROOT_ID.equals(victimId); // root deletion is prevented in earlier stages. @@ -1186,7 +1188,9 @@ public class IgfsMetaManager extends IgfsManager { if (allIds.remove(null)) return null; // A fragment of the path no longer exists. - boolean added = allIds.add(IgfsUtils.TRASH_ID); + IgniteUuid trashId = IgfsUtils.randomTrashId(); + + boolean added = allIds.add(trashId); assert added; final IgniteInternalTx tx = startTx(); @@ -1205,7 +1209,7 @@ public class IgfsMetaManager extends IgfsManager { throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " + "empty and recursive flag is not set)."); - IgfsFileInfo destInfo = infoMap.get(IgfsUtils.TRASH_ID); + IgfsFileInfo destInfo = infoMap.get(trashId); assert destInfo != null; @@ -1233,7 +1237,7 @@ public class IgfsMetaManager extends IgfsManager { id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId())); // Add listing entry into the destination parent listing. - id2InfoPrj.invoke(IgfsUtils.TRASH_ID, new ListingAdd(destFileName, srcEntry)); + id2InfoPrj.invoke(trashId, new ListingAdd(destFileName, srcEntry)); if (victimInfo.isFile()) // Update a file info of the removed file with a file path, @@ -1265,10 +1269,12 @@ public class IgfsMetaManager extends IgfsManager { * @param parentId Parent ID. * @param name Path name. * @param id Path ID. + * @param trashId Trash ID. * @return ID of an entry located directly under the trash directory. * @throws IgniteCheckedException If failed. */ - @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id) + @Nullable private IgniteUuid softDeleteNonTx(@Nullable IgniteUuid parentId, @Nullable String name, IgniteUuid id, + IgniteUuid trashId) throws IgniteCheckedException { assert validTxState(true); @@ -1284,8 +1290,7 @@ public class IgfsMetaManager extends IgfsManager { return null; // Root was never created. // Ensure trash directory existence. - if (id2InfoPrj.get(IgfsUtils.TRASH_ID) == null) - id2InfoPrj.getAndPut(IgfsUtils.TRASH_ID, new IgfsFileInfo(IgfsUtils.TRASH_ID)); + createSystemEntryIfAbsent(trashId); Map rootListing = rootInfo.listing(); @@ -1310,7 +1315,7 @@ public class IgfsMetaManager extends IgfsManager { id2InfoPrj.getAndPut(newInfo.id(), newInfo); // Add new info to trash listing. - id2InfoPrj.invoke(IgfsUtils.TRASH_ID, new ListingAdd(newInfo.id().toString(), + id2InfoPrj.invoke(trashId, new ListingAdd(newInfo.id().toString(), new IgfsListingEntry(newInfo))); // Remove listing entries from root. @@ -1324,10 +1329,9 @@ public class IgfsMetaManager extends IgfsManager { } else { // Ensure trash directory existence. - if (id2InfoPrj.get(IgfsUtils.TRASH_ID) == null) - id2InfoPrj.getAndPut(IgfsUtils.TRASH_ID, new IgfsFileInfo(IgfsUtils.TRASH_ID)); + createSystemEntryIfAbsent(trashId); - moveNonTx(id, name, parentId, id.toString(), IgfsUtils.TRASH_ID); + moveNonTx(id, name, parentId, id.toString(), trashId); resId = id; } @@ -1500,21 +1504,24 @@ public class IgfsMetaManager extends IgfsManager { public Collection pendingDeletes() throws IgniteCheckedException { if (busyLock.enterBusy()) { try { - IgfsFileInfo trashInfo = id2InfoPrj.get(IgfsUtils.TRASH_ID); + Collection ids = new HashSet<>(); - if (trashInfo != null) { - Map listing = trashInfo.listing(); + for (int i = 0; i < IgfsUtils.TRASH_CONCURRENCY; i++) { + IgniteUuid trashId = IgfsUtils.trashId(i); - if (listing != null && !listing.isEmpty()) { - return F.viewReadOnly(listing.values(), new IgniteClosure() { - @Override public IgniteUuid apply(IgfsListingEntry e) { - return e.fileId(); - } - }); + IgfsFileInfo trashInfo = id2InfoPrj.get(trashId); + + if (trashInfo != null) { + Map listing = trashInfo.listing(); + + if (listing != null && !listing.isEmpty()) { + for (IgfsListingEntry entry : listing.values()) + ids.add(entry.fileId()); + } } } - return Collections.emptySet(); + return ids; } finally { busyLock.leaveBusy(); @@ -2451,6 +2458,8 @@ public class IgfsMetaManager extends IgfsManager { assert fs != null; assert path != null; + final IgniteUuid trashId = IgfsUtils.randomTrashId(); + SynchronizationTask task = new SynchronizationTask() { @Override public Boolean onSuccess(Map infos) throws Exception { IgfsFileInfo info = infos.get(path); @@ -2464,12 +2473,12 @@ public class IgfsMetaManager extends IgfsManager { if (path.parent() != null) { assert infos.containsKey(path.parent()); - softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id()); + softDeleteNonTx(infos.get(path.parent()).id(), path.name(), info.id(), trashId); } else { assert IgfsUtils.ROOT_ID.equals(info.id()); - softDeleteNonTx(null, path.name(), info.id()); + softDeleteNonTx(null, path.name(), info.id(), trashId); } // Update the deleted file info with path information for delete worker. @@ -2487,7 +2496,7 @@ public class IgfsMetaManager extends IgfsManager { } }; - Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(IgfsUtils.TRASH_ID), path); + Boolean res = synchronizeAndExecute(task, fs, false, Collections.singleton(trashId), path); delWorker.signal(); @@ -3437,6 +3446,8 @@ public class IgfsMetaManager extends IgfsManager { DirectoryChainBuilder b = null; + IgniteUuid trashId = IgfsUtils.randomTrashId(); + while (true) { if (busyLock.enterBusy()) { try { @@ -3457,11 +3468,11 @@ public class IgfsMetaManager extends IgfsManager { if (overwrite) // Lock also the TRASH directory because in case of overwrite we // may need to delete the old file: - b.idSet.add(IgfsUtils.TRASH_ID); + b.idSet.add(trashId); final Map lockedInfos = lockIds(b.idSet); - assert !overwrite || lockedInfos.get(IgfsUtils.TRASH_ID) != null; // TRASH must exist at this point. + assert !overwrite || lockedInfos.get(trashId) != null; // TRASH must exist at this point. // If the path was changed, we close the current Tx and repeat the procedure again // starting from taking the path ids. @@ -3532,8 +3543,8 @@ public class IgfsMetaManager extends IgfsManager { id2InfoPrj.invoke(parentId, new ListingRemove(name, deletedEntry.fileId())); // Add listing entry into the destination parent listing. - id2InfoPrj.invoke(IgfsUtils.TRASH_ID, new ListingAdd( - lowermostExistingInfo.id().toString(), deletedEntry)); + id2InfoPrj.invoke(trashId, + new ListingAdd(lowermostExistingInfo.id().toString(), deletedEntry)); // Update a file info of the removed file with a file path, // which will be used by delete worker for event notifications. http://git-wip-us.apache.org/repos/asf/ignite/blob/421d0c1d/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 a3c0c21..6f8960a 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 @@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.igfs; import java.lang.reflect.Constructor; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -53,16 +54,75 @@ public class IgfsUtils { /** ID for the root directory. */ public static final IgniteUuid ROOT_ID = new IgniteUuid(new UUID(0, 0), 0); - /** ID of the trash directory. */ - public static final IgniteUuid TRASH_ID = new IgniteUuid(new UUID(0, 1), 0); - /** Lock Id used to lock files being deleted from TRASH. This is a global constant. */ - public static final IgniteUuid DELETE_LOCK_ID = new IgniteUuid(new UUID(0L, 0L), 0L); + public static final IgniteUuid DELETE_LOCK_ID = new IgniteUuid(new UUID(0, 0), 0); + + /** Constant trash concurrency level. */ + public static final int TRASH_CONCURRENCY = 16; + + /** Trash directory IDs. */ + private static final IgniteUuid[] TRASH_IDS; /** Maximum number of file unlock transaction retries when topology changes. */ private static final int MAX_CACHE_TX_RETRIES = IgniteSystemProperties.getInteger(IGNITE_CACHE_RETRIES_COUNT, 100); /** + * Static initializer. + */ + static { + TRASH_IDS = new IgniteUuid[TRASH_CONCURRENCY]; + + for (int i = 0; i < TRASH_CONCURRENCY; i++) + TRASH_IDS[i] = new IgniteUuid(new UUID(0, i + 1), 0); + } + + /** + * Get random trash ID. + * + * @return Trash ID. + */ + public static IgniteUuid randomTrashId() { + return TRASH_IDS[ThreadLocalRandom.current().nextInt(TRASH_CONCURRENCY)]; + } + + /** + * Get trash ID for the given index. + * + * @param idx Index. + * @return Trahs ID. + */ + public static IgniteUuid trashId(int idx) { + assert idx >= 0 && idx < TRASH_CONCURRENCY; + + return TRASH_IDS[idx]; + } + + /** + * Check whether provided ID is trash ID. + * + * @param id ID. + * @return {@code True} if this is trash ID. + */ + public static boolean isTrashId(IgniteUuid id) { + assert id != null; + + UUID gid = id.globalId(); + + return id.localId() == 0 && gid.getMostSignificantBits() == 0 && + gid.getLeastSignificantBits() > 0 && gid.getLeastSignificantBits() <= TRASH_CONCURRENCY; + } + + /** + * Check whether provided ID is either root ID or trash ID. + * + * @param id ID. + * @return {@code True} if this is root ID or trash ID. + */ + public static boolean isRootOrTrashId(IgniteUuid id) { + return ROOT_ID.equals(id) || isTrashId(id); + } + + /** * Converts any passed exception to IGFS exception. * * @param err Initial exception. http://git-wip-us.apache.org/repos/asf/ignite/blob/421d0c1d/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 015b992..52d8bd5 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 @@ -3138,14 +3138,22 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { igfs.format(); int prevDifferentSize = Integer.MAX_VALUE; // Previous different size. - int size; int constCnt = 0, totalCnt = 0; final int constThreshold = 20; final long sleepPeriod = 500L; final long totalThreshold = CACHE_EMPTY_TIMEOUT / sleepPeriod; while (true) { - size = sumCacheSize(igfs); + int metaSize = 0; + + for (IgniteUuid metaId : getMetaCache(igfs).keySet()) { + if (!IgfsUtils.isRootOrTrashId(metaId)) + metaSize++; + } + + int dataSize = getDataCache(igfs).size(); + + int size = metaSize + dataSize; if (size <= 2) return; // Caches are cleared, we're done. (2 because ROOT & TRASH always exist). @@ -3206,15 +3214,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest { } /** - * Gets summary IGFS cache size. - * @param igfs The IGFS to measure. - * @return data cache size + meta cache size. - */ - private static int sumCacheSize(IgniteFileSystem igfs) { - return getMetaCache(igfs).size() + getDataCache(igfs).size(); - } - - /** * Clear particular {@link UniversalFileSystemAdapter}. * * @param uni IGFS. http://git-wip-us.apache.org/repos/asf/ignite/blob/421d0c1d/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java index 5f806f7..75e7195 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSizeSelfTest.java @@ -536,6 +536,7 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest { }).start(); // Now add file ID to trash listing so that delete worker could "see" it. + IgniteUuid trashId = IgfsUtils.trashId(0); try (Transaction tx = metaCache.unwrap(Ignite.class).transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { Map listing = Collections.singletonMap(path.name(), @@ -545,17 +546,17 @@ public class IgfsSizeSelfTest extends IgfsCommonAbstractTest { metaCache.put(IgfsUtils.ROOT_ID, new IgfsFileInfo(IgfsUtils.ROOT_ID)); // Add file to trash listing. - IgfsFileInfo trashInfo = metaCache.get(IgfsUtils.TRASH_ID); + IgfsFileInfo trashInfo = metaCache.get(trashId); if (trashInfo == null) - metaCache.put(IgfsUtils.TRASH_ID, new IgfsFileInfo(listing, new IgfsFileInfo(IgfsUtils.TRASH_ID))); + metaCache.put(trashId, new IgfsFileInfo(listing, new IgfsFileInfo(trashId))); else - metaCache.put(IgfsUtils.TRASH_ID, new IgfsFileInfo(listing, trashInfo)); + metaCache.put(trashId, new IgfsFileInfo(listing, trashInfo)); tx.commit(); } - assert metaCache.get(IgfsUtils.TRASH_ID) != null; + assert metaCache.get(trashId) != null; // Now the file is locked and is located in trash, try adding some more data. os = igfs.create(otherPath, false);