You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2016/03/22 17:49:55 UTC

[19/50] [abbrv] ignite git commit: IGNITE-2838: IGFS: Opimized format of IgfsListingEntry. Now it contains only file ID and boolean flag endicating whether this a directory or file.

IGNITE-2838: IGFS: Opimized format of IgfsListingEntry. Now it contains only file ID and boolean flag endicating whether this a directory or file.


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

Branch: refs/heads/ignite-2004
Commit: 7b7233efeeacc0c65833dec925a73906d9880608
Parents: 18de768
Author: thatcoach <pp...@list.ru>
Authored: Tue Mar 15 20:46:13 2016 +0300
Committer: thatcoach <pp...@list.ru>
Committed: Tue Mar 15 20:46:13 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsFileImpl.java  |  29 ---
 .../internal/processors/igfs/IgfsFileInfo.java  |  38 +++-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsListingEntry.java       | 129 ++---------
 .../processors/igfs/IgfsMetaManager.java        | 221 +++----------------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 -
 6 files changed, 88 insertions(+), 339 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
index 4a96e81..3576a06 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileImpl.java
@@ -124,35 +124,6 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable {
         modificationTime = info.modificationTime();
     }
 
-    /**
-     * Constructs file instance.
-     *
-     * @param path Path.
-     * @param entry Listing entry.
-     */
-    public IgfsFileImpl(IgfsPath path, IgfsListingEntry entry, long globalGrpSize) {
-        A.notNull(path, "path");
-        A.notNull(entry, "entry");
-
-        this.path = path;
-        fileId = entry.fileId();
-
-        blockSize = entry.blockSize();
-
-        // By contract file must have blockSize > 0, while directory's blockSize == 0:
-        assert entry.isFile() == (blockSize > 0);
-        assert entry.isDirectory() == (blockSize == 0);
-
-        grpBlockSize = entry.affinityKey() == null ? globalGrpSize :
-            entry.length() == 0 ? globalGrpSize : entry.length();
-
-        len = entry.length();
-        props = entry.properties();
-
-        accessTime = entry.accessTime();
-        modificationTime = entry.modificationTime();
-    }
-
     /** {@inheritDoc} */
     @Override public IgfsPath path() {
         return path;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 0a85657..ba484bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -297,9 +297,6 @@ public final class IgfsFileInfo implements Externalizable {
         this.props = props == null || props.isEmpty() ? null :
             cpProps ? new GridLeanMap<>(props) : props;
 
-        if (listing == null && isDir)
-            this.listing = Collections.emptyMap();
-
         this.lockId = lockId;
         this.evictExclude = evictExclude;
     }
@@ -410,13 +407,38 @@ public final class IgfsFileInfo implements Externalizable {
      * @return Directory listing.
      */
     public Map<String, IgfsListingEntry> listing() {
-        // Always wrap into unmodifiable map to be able to avoid illegal modifications in order pieces of the code.
-        if (isFile())
-            return Collections.unmodifiableMap(Collections.<String, IgfsListingEntry>emptyMap());
+        return listing != null ? listing : Collections.<String, IgfsListingEntry>emptyMap();
+    }
+
+    /**
+     * @return {@code True} if at least one child exists.
+     */
+    public boolean hasChildren() {
+        return !F.isEmpty(listing);
+    }
+
+    /**
+     * @param name Child name.
+     * @return {@code True} if child with such name exists.
+     */
+    public boolean hasChild(String name) {
+        return listing != null && listing.containsKey(name);
+    }
 
-        assert listing != null;
+    /**
+     * @param name Child name.
+     * @param expId Expected child ID.
+     * @return {@code True} if child with such name exists.
+     */
+    public boolean hasChild(String name, IgniteUuid expId) {
+        if (listing != null) {
+            IgfsListingEntry entry = listing.get(name);
+
+            if (entry != null)
+                return F.eq(expId, entry.fileId());
+        }
 
-        return Collections.unmodifiableMap(listing);
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 0e52927..f44eda8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -870,9 +870,13 @@ public final class IgfsImpl implements IgfsEx {
 
                         // Perform the listing.
                         for (Map.Entry<String, IgfsListingEntry> e : info.listing().entrySet()) {
-                            IgfsPath p = new IgfsPath(path, e.getKey());
+                            IgfsFileInfo childInfo = meta.info(e.getValue().fileId());
 
-                            files.add(new IgfsFileImpl(p, e.getValue(), data.groupBlockSize()));
+                            if (childInfo != null) {
+                                IgfsPath childPath = new IgfsPath(path, e.getKey());
+
+                                files.add(new IgfsFileImpl(childPath, childInfo, data.groupBlockSize()));
+                            }
                         }
                     }
                 } else if (mode == PRIMARY) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
index 1cdc8a9..61d9265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsListingEntry.java
@@ -21,7 +21,8 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
-import java.util.Map;
+
+import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -34,25 +35,10 @@ public class IgfsListingEntry implements Externalizable {
     private static final long serialVersionUID = 0L;
 
     /** File id. */
-    private IgniteUuid fileId;
-
-    /** File affinity key. */
-    private IgniteUuid affKey;
-
-    /** Positive block size if file, 0 if directory. */
-    private int blockSize;
-
-    /** File length. */
-    private long len;
-
-    /** Last access time. */
-    private long accessTime;
+    private IgniteUuid id;
 
-    /** Last modification time. */
-    private long modificationTime;
-
-    /** File properties. */
-    private Map<String, String> props;
+    /** Directory marker. */
+    private boolean dir;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -65,132 +51,51 @@ public class IgfsListingEntry implements Externalizable {
      * @param fileInfo File info to construct listing entry from.
      */
     public IgfsListingEntry(IgfsFileInfo fileInfo) {
-        fileId = fileInfo.id();
-        affKey = fileInfo.affinityKey();
-
-        if (fileInfo.isFile()) {
-            blockSize = fileInfo.blockSize();
-            len = fileInfo.length();
-        }
-
-        props = fileInfo.properties();
-        accessTime = fileInfo.accessTime();
-        modificationTime = fileInfo.modificationTime();
-    }
-
-    /**
-     * Creates listing entry with updated length.
-     *
-     * @param entry Entry.
-     * @param len New length.
-     */
-    public IgfsListingEntry(IgfsListingEntry entry, long len, long accessTime, long modificationTime) {
-        fileId = entry.fileId;
-        affKey = entry.affKey;
-        blockSize = entry.blockSize;
-        props = entry.props;
-        this.accessTime = accessTime;
-        this.modificationTime = modificationTime;
-
-        this.len = len;
+        id = fileInfo.id();
+        dir = fileInfo.isDirectory();
     }
 
     /**
      * @return Entry file ID.
      */
     public IgniteUuid fileId() {
-        return fileId;
-    }
-
-    /**
-     * @return File affinity key, if specified.
-     */
-    public IgniteUuid affinityKey() {
-        return affKey;
+        return id;
     }
 
     /**
      * @return {@code True} if entry represents file.
      */
     public boolean isFile() {
-        return blockSize > 0;
+        return !dir;
     }
 
     /**
      * @return {@code True} if entry represents directory.
      */
     public boolean isDirectory() {
-        return blockSize == 0;
-    }
-
-    /**
-     * @return Block size.
-     */
-    public int blockSize() {
-        return blockSize;
-    }
-
-    /**
-     * @return Length.
-     */
-    public long length() {
-        return len;
-    }
-
-    /**
-     * @return Last access time.
-     */
-    public long accessTime() {
-        return accessTime;
-    }
-
-    /**
-     * @return Last modification time.
-     */
-    public long modificationTime() {
-        return modificationTime;
-    }
-
-    /**
-     * @return Properties map.
-     */
-    public Map<String, String> properties() {
-        return props;
+        return dir;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeGridUuid(out, fileId);
-        out.writeInt(blockSize);
-        out.writeLong(len);
-        U.writeStringMap(out, props);
-        out.writeLong(accessTime);
-        out.writeLong(modificationTime);
+        U.writeGridUuid(out, id);
+        out.writeBoolean(dir);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        fileId = U.readGridUuid(in);
-        blockSize = in.readInt();
-        len = in.readLong();
-        props = U.readStringMap(in);
-        accessTime = in.readLong();
-        modificationTime = in.readLong();
+        id = U.readGridUuid(in);
+        dir = in.readBoolean();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o) return true;
-        if (!(o instanceof IgfsListingEntry)) return false;
-
-        IgfsListingEntry that = (IgfsListingEntry)o;
-
-        return fileId.equals(that.fileId);
+    @Override public boolean equals(Object other) {
+        return this == other || other instanceof IgfsListingEntry && F.eq(id, ((IgfsListingEntry)other).id);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        return fileId.hashCode();
+        return id.hashCode();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/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 d3aae58..463e7a8 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
@@ -801,18 +801,12 @@ public class IgfsMetaManager extends IgfsManager {
         if (!parentInfo.isDirectory())
             throw fsException(new IgfsPathIsNotDirectoryException("Parent file is not a directory: " + parentInfo));
 
-        Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+        IgfsListingEntry childEntry = parentInfo.listing().get(fileName);
 
-        assert parentListing != null;
+        if (childEntry != null)
+            return childEntry.fileId();
 
-        IgfsListingEntry entry = parentListing.get(fileName);
-
-        if (entry != null)
-            return entry.fileId();
-
-        IgniteUuid fileId = newFileInfo.id();
-
-        if (!id2InfoPrj.putIfAbsent(fileId, newFileInfo))
+        if (!id2InfoPrj.putIfAbsent(newFileInfo.id(), newFileInfo))
             throw fsException("Failed to add file details into cache: " + newFileInfo);
 
         id2InfoPrj.invoke(parentId, new ListingAdd(fileName, new IgfsListingEntry(newFileInfo)));
@@ -915,7 +909,7 @@ public class IgfsMetaManager extends IgfsManager {
                     assert dstTargetInfo.isDirectory();
 
                     // 7. Last check: does destination target already have listing entry with the same name?
-                    if (dstTargetInfo.listing().containsKey(dstName)) {
+                    if (dstTargetInfo.hasChild(dstName)) {
                         throw new IgfsPathAlreadyExistsException("Failed to perform move because destination already " +
                             "contains entry with the same name existing file [src=" + srcPath +
                             ", dst=" + dstPath + ']');
@@ -973,14 +967,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                 // If parent info is null, it doesn't exist.
                 if (parentInfo != null) {
-                    IgfsListingEntry childEntry = parentInfo.listing().get(pathParts.get(i));
-
-                    // If expected child exists.
-                    if (childEntry != null) {
-                        // If child ID matches expected ID.
-                        if (F.eq(childEntry.fileId(), expIds.get(i + 1)))
-                            continue;
-                    }
+                    if (parentInfo.hasChild(pathParts.get(i), expIds.get(i + 1)))
+                        continue;
                 }
             }
 
@@ -1046,7 +1034,6 @@ public class IgfsMetaManager extends IgfsManager {
                 fileId + ']'));
 
         IgfsListingEntry srcEntry = srcInfo.listing().get(srcFileName);
-        IgfsListingEntry destEntry = destInfo.listing().get(destFileName);
 
         // If source file does not exist or was re-created.
         if (srcEntry == null || !srcEntry.fileId().equals(fileId))
@@ -1055,10 +1042,10 @@ public class IgfsMetaManager extends IgfsManager {
                 ", srcParentId=" + srcParentId + ", srcEntry=" + srcEntry + ']'));
 
         // If stored file already exist.
-        if (destEntry != null)
+        if (destInfo.hasChild(destFileName))
             throw fsException(new IgfsPathAlreadyExistsException("Failed to add file name into the destination " +
                 " directory (file already exists) [fileId=" + fileId + ", destFileName=" + destFileName +
-                ", destParentId=" + destParentId + ", destEntry=" + destEntry + ']'));
+                ", destParentId=" + destParentId + ']'));
 
         // Remove listing entry from the source parent listing.
         id2InfoPrj.invoke(srcParentId, new ListingRemove(srcFileName, srcEntry.fileId()));
@@ -1174,7 +1161,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     final IgfsFileInfo victimInfo = infoMap.get(victimId);
 
-                    if (!recursive && victimInfo.isDirectory() && !victimInfo.listing().isEmpty())
+                    if (!recursive && victimInfo.hasChildren())
                         // Throw exception if not empty and not recursive.
                         throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
                             "empty and recursive flag is not set).");
@@ -1187,7 +1174,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                     final String destFileName = victimId.toString();
 
-                    assert destInfo.listing().get(destFileName) == null : "Failed to add file name into the " +
+                    assert !destInfo.hasChild(destFileName) : "Failed to add file name into the " +
                         "destination directory (file already exists) [destName=" + destFileName + ']';
 
                     IgfsFileInfo srcParentInfo = infoMap.get(pathIdList.get(pathIdList.size() - 2));
@@ -1338,8 +1325,8 @@ public class IgfsMetaManager extends IgfsManager {
 
                     int i = 1;
 
-                    for (IgfsListingEntry entry : listing.values())
-                        allIds[i++] = entry.fileId();
+                    for (IgfsListingEntry childEntry : listing.values())
+                        allIds[i++] = childEntry.fileId();
 
                     Map<IgniteUuid, IgfsFileInfo> locks = lockIds(allIds);
 
@@ -1347,35 +1334,37 @@ public class IgfsMetaManager extends IgfsManager {
 
                     // Ensure parent is still in place.
                     if (parentInfo != null) {
-                        Map<String, IgfsListingEntry> newListing =
-                            new HashMap<>(parentInfo.listing().size(), 1.0f);
+                        Map<String, IgfsListingEntry> parentListing = parentInfo.listing();
+
+                        Map<String, IgfsListingEntry> newListing = new HashMap<>(parentListing.size(), 1.0f);
 
-                        newListing.putAll(parentInfo.listing());
+                        newListing.putAll(parentListing);
 
                         // Remove child entries if possible.
                         for (Map.Entry<String, IgfsListingEntry> entry : listing.entrySet()) {
-                            IgniteUuid entryId = entry.getValue().fileId();
+                            String childName = entry.getKey();
+                            IgniteUuid childId = entry.getValue().fileId();
 
-                            IgfsFileInfo entryInfo = locks.get(entryId);
+                            IgfsFileInfo entryInfo = locks.get(childId);
 
                             if (entryInfo != null) {
                                 // File must be locked for deletion:
                                 assert entryInfo.isDirectory() || IgfsUtils.DELETE_LOCK_ID.equals(entryInfo.lockId());
 
                                 // Delete only files or empty folders.
-                                if (entryInfo.isFile() || entryInfo.isDirectory() && entryInfo.listing().isEmpty()) {
-                                    id2InfoPrj.getAndRemove(entryId);
+                                if (!entryInfo.hasChildren()) {
+                                    id2InfoPrj.getAndRemove(childId);
 
-                                    newListing.remove(entry.getKey());
+                                    newListing.remove(childName);
 
-                                    res.add(entryId);
+                                    res.add(childId);
                                 }
                             }
                             else {
                                 // Entry was deleted concurrently.
-                                newListing.remove(entry.getKey());
+                                newListing.remove(childName);
 
-                                res.add(entryId);
+                                res.add(childId);
                             }
                         }
 
@@ -1436,10 +1425,10 @@ public class IgfsMetaManager extends IgfsManager {
 
                         assert parentInfo != null;
 
-                        IgfsListingEntry listingEntry = parentInfo.listing().get(name);
+                        IgfsListingEntry childEntry = parentInfo.listing().get(name);
 
-                        if (listingEntry != null)
-                            id2InfoPrj.invoke(parentId, new ListingRemove(name, listingEntry.fileId()));
+                        if (childEntry != null)
+                            id2InfoPrj.invoke(parentId, new ListingRemove(name, childEntry.fileId()));
 
                         IgfsFileInfo deleted = id2InfoPrj.getAndRemove(id);
 
@@ -1481,13 +1470,9 @@ public class IgfsMetaManager extends IgfsManager {
 
                     IgfsFileInfo trashInfo = id2InfoPrj.get(trashId);
 
-                    if (trashInfo != null) {
-                        Map<String, IgfsListingEntry> listing = trashInfo.listing();
-
-                        if (listing != null && !listing.isEmpty()) {
-                            for (IgfsListingEntry entry : listing.values())
-                                ids.add(entry.fileId());
-                        }
+                    if (trashInfo != null && trashInfo.hasChildren()) {
+                        for (IgfsListingEntry entry : trashInfo.listing().values())
+                            ids.add(entry.fileId());
                     }
                 }
 
@@ -1543,11 +1528,7 @@ public class IgfsMetaManager extends IgfsManager {
                 return null; // File not found.
 
             if (parentInfo != null) {
-                Map<String, IgfsListingEntry> listing = parentInfo.listing();
-
-                IgfsListingEntry entry = listing.get(fileName);
-
-                if (entry == null || !entry.fileId().equals(fileId)) // File was removed or recreated.
+                if (!parentInfo.hasChild(fileName, fileId)) // File was removed or recreated.
                     return null;
             }
 
@@ -1622,35 +1603,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Asynchronously updates record in parent listing.
-     *
-     * @param parentId Parent ID.
-     * @param fileId File ID.
-     * @param fileName File name.
-     * @param lenDelta Length delta.
-     * @param modificationTime Last modification time.
-     */
-    public void updateParentListingAsync(IgniteUuid parentId, IgniteUuid fileId, String fileName, long lenDelta,
-        long modificationTime) {
-        if (busyLock.enterBusy()) {
-            try {
-                assert parentId != null;
-
-                assert validTxState(false);
-
-                id2InfoPrj.invokeAsync(parentId, new UpdateListingEntry(fileId, fileName, lenDelta, -1,
-                    modificationTime));
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to update parent listing because Grid is stopping [parentId=" +
-                parentId + ", fileId=" + fileId + ", fileName=" + fileName + ']');
-    }
-
-    /**
      * Reserve space for file.
      *
      * @param path File path.
@@ -2980,10 +2932,8 @@ public class IgfsMetaManager extends IgfsManager {
                         throw fsException(new IgfsPathNotFoundException("Failed to update times " +
                             "(parent was not found): " + fileName));
 
-                    IgfsListingEntry entry = parentInfo.listing().get(fileName);
-
                     // Validate listing.
-                    if (entry == null || !entry.fileId().equals(fileId))
+                    if (!parentInfo.hasChild(fileName, fileId))
                         throw fsException(new IgfsConcurrentModificationException("Failed to update times " +
                                 "(file concurrently modified): " + fileName));
 
@@ -2994,9 +2944,6 @@ public class IgfsMetaManager extends IgfsManager {
                         modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
                     );
 
-                    id2InfoPrj.invoke(parentId, new UpdateListingEntry(fileId, fileName, 0, accessTime,
-                        modificationTime));
-
                     tx.commit();
                 }
                 finally {
@@ -3143,100 +3090,6 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
-     * Updates file length information in parent listing.
-     */
-    private static final class UpdateListingEntry implements EntryProcessor<IgniteUuid, IgfsFileInfo, Void>,
-        Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** File name. */
-        private String fileName;
-
-        /** File id. */
-        private IgniteUuid fileId;
-
-        /** Length delta. */
-        private long lenDelta;
-
-        /** Last access time. */
-        private long accessTime;
-
-        /** Last modification time. */
-        private long modificationTime;
-
-        /**
-         * Empty constructor required by {@link Externalizable}.
-         */
-        public UpdateListingEntry() {
-            // No-op.
-        }
-
-        /**
-         * @param fileId Expected file id in parent directory listing.
-         * @param fileName File name.
-         * @param lenDelta Length delta.
-         * @param accessTime Last access time.
-         * @param modificationTime Last modification time.
-         */
-        private UpdateListingEntry(IgniteUuid fileId,
-            String fileName,
-            long lenDelta,
-            long accessTime,
-            long modificationTime) {
-            this.fileId = fileId;
-            this.fileName = fileName;
-            this.lenDelta = lenDelta;
-            this.accessTime = accessTime;
-            this.modificationTime = modificationTime;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<IgniteUuid, IgfsFileInfo> e, Object... args) {
-            IgfsFileInfo fileInfo = e.getValue();
-
-            Map<String, IgfsListingEntry> listing = fileInfo.listing();
-
-            IgfsListingEntry entry = listing.get(fileName);
-
-            if (entry == null || !entry.fileId().equals(fileId))
-                return null;
-
-            entry = new IgfsListingEntry(entry, entry.length() + lenDelta,
-                accessTime == -1 ? entry.accessTime() : accessTime,
-                modificationTime == -1 ? entry.modificationTime() : modificationTime);
-
-            // Create new map to replace info.
-            listing = new HashMap<>(listing);
-
-            // Modify listing map in-place since map is serialization-safe.
-            listing.put(fileName, entry);
-
-            e.setValue(new IgfsFileInfo(listing, fileInfo));
-
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            U.writeGridUuid(out, fileId);
-            out.writeUTF(fileName);
-            out.writeLong(lenDelta);
-            out.writeLong(accessTime);
-            out.writeLong(modificationTime);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException {
-            fileId = U.readGridUuid(in);
-            fileName = in.readUTF();
-            lenDelta = in.readLong();
-            accessTime = in.readLong();
-            modificationTime = in.readLong();
-        }
-    }
-
-    /**
      * Remove entry from directory listing.
      */
     @GridInternal
@@ -3608,13 +3461,9 @@ public class IgfsMetaManager extends IgfsManager {
                                 throw new IgfsParentNotDirectoryException("Failed to " + (append ? "open" : "create" )
                                     + " file (parent element is not a directory)");
 
-                            Map<String, IgfsListingEntry> parentListing = lowermostExistingInfo.listing();
-
                             final String uppermostFileToBeCreatedName = b.components.get(b.existingIdCnt - 1);
 
-                            final IgfsListingEntry entry = parentListing.get(uppermostFileToBeCreatedName);
-
-                            if (entry == null) {
+                            if (!lowermostExistingInfo.hasChild(uppermostFileToBeCreatedName)) {
                                 b.doBuild();
 
                                 assert b.leafInfo != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b7233ef/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 8c11073..3bf1011 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
@@ -381,8 +381,6 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
                     throw new IOException("File to read file metadata: " + fileInfo.path(), e);
                 }
 
-                meta.updateParentListingAsync(parentId, fileInfo.id(), fileName, bytes, modificationTime);
-
                 if (err != null)
                     throw err;
             }