You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/03/15 14:35:21 UTC

[12/16] ignite git commit: Dramatically simplified IgfsListingEntry.

Dramatically simplified IgfsListingEntry.


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

Branch: refs/heads/ignite-2813
Commit: 2985289e2dafd376fba602a979260ed627557eac
Parents: 063e72f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 15 16:18:24 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 15 16:18:24 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsFileImpl.java  |  29 -----
 .../processors/igfs/IgfsListingEntry.java       | 129 +++----------------
 .../processors/igfs/IgfsMetaManager.java        | 126 ------------------
 .../processors/igfs/IgfsOutputStreamImpl.java   |   2 -
 4 files changed, 17 insertions(+), 269 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2985289e/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/2985289e/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/2985289e/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 aee1191..59874fb 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
@@ -1607,35 +1607,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.
@@ -2977,9 +2948,6 @@ public class IgfsMetaManager extends IgfsManager {
                         modificationTime == -1 ? fileInfo.modificationTime() : modificationTime)
                     );
 
-                    id2InfoPrj.invoke(parentId, new UpdateListingEntry(fileId, fileName, 0, accessTime,
-                        modificationTime));
-
                     tx.commit();
                 }
                 finally {
@@ -3126,100 +3094,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

http://git-wip-us.apache.org/repos/asf/ignite/blob/2985289e/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;
             }