You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/02/26 13:12:40 UTC

[12/17] ignite git commit: IGNITE-2352: IGFS: Correct access time and modification time propagation from secondary file system. This closes #501.

IGNITE-2352: IGFS: Correct access time and modification time propagation from secondary file system. This closes #501.


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

Branch: refs/heads/ignite-1232
Commit: 52e178f6e0c3ca5930713c69b34364994ad0ccfe
Parents: dee6190
Author: iveselovskiy <iv...@gridgain.com>
Authored: Thu Feb 25 14:38:22 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Feb 25 14:38:22 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsFileImpl.java  | 23 ++++++
 .../internal/processors/igfs/IgfsFileInfo.java  | 54 +++++++-------
 .../internal/processors/igfs/IgfsImpl.java      | 23 +++---
 .../processors/igfs/IgfsMetaManager.java        | 23 ++++--
 .../igfs/IgfsDataManagerSelfTest.java           | 24 +++++--
 .../igfs/IgfsDualAbstractSelfTest.java          | 74 +++++++++++++++++++-
 .../processors/igfs/IgfsFileInfoSelfTest.java   | 11 ++-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  7 +-
 8 files changed, 179 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/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 5d7389b..4a96e81 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
@@ -70,6 +70,29 @@ public final class IgfsFileImpl implements IgfsFile, Externalizable {
     }
 
     /**
+     * A copy constructor. All the fields are copied from the copied {@code igfsFile}, but the {@code groupBlockSize}
+     * which is specified separately.
+     *
+     * @param igfsFile The file to copy.
+     */
+    public IgfsFileImpl(IgfsFile igfsFile, long grpBlockSize) {
+        A.notNull(igfsFile, "igfsFile");
+
+        this.path = igfsFile.path();
+        this.fileId = igfsFile instanceof IgfsFileImpl ? ((IgfsFileImpl)igfsFile).fileId : IgniteUuid.randomUuid();
+
+        this.blockSize = igfsFile.blockSize();
+        this.len = igfsFile.length();
+
+        this.grpBlockSize = igfsFile.isFile() ? grpBlockSize : 0L;
+
+        this.props = igfsFile.properties();
+
+        this.accessTime = igfsFile.accessTime();
+        this.modificationTime = igfsFile.modificationTime();
+    }
+
+    /**
      * Constructs directory info.
      *
      * @param path Path.

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/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 fa79d80..a69920b 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
@@ -47,6 +47,9 @@ public final class IgfsFileInfo implements Externalizable {
     /** ID of the trash directory. */
     public static final IgniteUuid TRASH_ID = new IgniteUuid(new UUID(0, 1), 0);
 
+    /** Special access time value, indicating that the modification time value should be taken. */
+    private static final long ACCESS_TIME_TAKE_MODIFICATION_TIME = -1L;
+
     /** Info ID. */
     private IgniteUuid id;
 
@@ -100,18 +103,22 @@ public final class IgfsFileInfo implements Externalizable {
      * @param id ID.
      */
     IgfsFileInfo(IgniteUuid id) {
-        this(true, id, 0, 0, null, null, null, null, false, System.currentTimeMillis(), false);
+        this(true, id, 0, 0, null, null, null, null, false, ACCESS_TIME_TAKE_MODIFICATION_TIME,
+            System.currentTimeMillis(), false);
     }
 
     /**
-     * Constructs directory or file info with {@link org.apache.ignite.configuration.FileSystemConfiguration#DFLT_BLOCK_SIZE default} block size.
+     * Constructs directory or file info with
+     * {@link org.apache.ignite.configuration.FileSystemConfiguration#DFLT_BLOCK_SIZE default} block size.
      *
      * @param isDir Constructs directory info if {@code true} or file info if {@code false}.
      * @param props Meta properties to set.
+     * @param accessTime The access time.
+     * @param modificationTime The modification time.
      */
-    public IgfsFileInfo(boolean isDir, @Nullable Map<String, String> props) {
+    public IgfsFileInfo(boolean isDir, @Nullable Map<String, String> props, long accessTime, long modificationTime) {
         this(isDir, null, isDir ? 0 : FileSystemConfiguration.DFLT_BLOCK_SIZE, 0, null, null, props, null, false,
-            System.currentTimeMillis(), false);
+            accessTime, modificationTime, false);
     }
 
     /**
@@ -120,7 +127,7 @@ public final class IgfsFileInfo implements Externalizable {
      * @param listing Listing.
      */
     IgfsFileInfo(Map<String, IgfsListingEntry> listing) {
-        this(true, null, 0, 0, null, listing, null, null, false, System.currentTimeMillis(), false);
+        this(listing, (Map<String,String>)null);
     }
 
     /**
@@ -130,20 +137,8 @@ public final class IgfsFileInfo implements Externalizable {
      * @param props The properties to set for the new directory.
      */
     IgfsFileInfo(@Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String,String> props) {
-        this(true/*dir*/, null, 0, 0, null, listing, props, null, false, System.currentTimeMillis(), false);
-    }
-
-    /**
-     * Constructs file info.
-     *
-     * @param blockSize Block size.
-     * @param affKey Affinity key.
-     * @param evictExclude Eviction exclude flag.
-     * @param props File properties.
-     */
-    IgfsFileInfo(int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude,
-        @Nullable Map<String, String> props) {
-        this(false, null, blockSize, 0, affKey, null, props, null, true, System.currentTimeMillis(), evictExclude);
+        this(true/*dir*/, null, 0, 0, null, listing, props, null, false, ACCESS_TIME_TAKE_MODIFICATION_TIME,
+            System.currentTimeMillis(), false);
     }
 
     /**
@@ -155,10 +150,13 @@ public final class IgfsFileInfo implements Externalizable {
      * @param lockId Lock ID.
      * @param props Properties.
      * @param evictExclude Evict exclude flag.
+     * @param accessTime The access time.
+     * @param modificationTime The modification time.
      */
     public IgfsFileInfo(int blockSize, long len, @Nullable IgniteUuid affKey, @Nullable IgniteUuid lockId,
-        boolean evictExclude, @Nullable Map<String, String> props) {
-        this(false, null, blockSize, len, affKey, null, props, lockId, true, System.currentTimeMillis(), evictExclude);
+        boolean evictExclude, @Nullable Map<String, String> props, long accessTime, long modificationTime) {
+        this(false, null, blockSize, len, affKey, null, props, lockId, true, accessTime, modificationTime,
+            evictExclude);
     }
 
     /**
@@ -202,10 +200,13 @@ public final class IgfsFileInfo implements Externalizable {
      * @param len Size of a file.
      * @param props File properties to set.
      * @param evictExclude Evict exclude flag.
+     * @param accessTime The access time.
+     * @param modificationTime The modification time.
      */
-    IgfsFileInfo(int blockSize, long len, boolean evictExclude, @Nullable Map<String, String> props) {
+    IgfsFileInfo(int blockSize, long len, boolean evictExclude, @Nullable Map<String, String> props,
+        long accessTime, long modificationTime) {
         this(blockSize == 0, // NB The contract is: (blockSize == 0) <=> isDirectory()
-            null, blockSize, len, null, null, props, null, true, System.currentTimeMillis(), evictExclude);
+            null, blockSize, len, null, null, props, null, true, accessTime, modificationTime, evictExclude);
     }
 
     /**
@@ -243,13 +244,14 @@ public final class IgfsFileInfo implements Externalizable {
      * @param props File properties.
      * @param lockId Lock ID.
      * @param cpProps Flag to copy properties map.
+     * @param accessTime The access time.
      * @param modificationTime Last modification time.
      * @param evictExclude Evict exclude flag.
      */
     private IgfsFileInfo(boolean isDir, @Nullable IgniteUuid id, int blockSize, long len, @Nullable IgniteUuid affKey,
         @Nullable Map<String, IgfsListingEntry> listing, @Nullable Map<String, String> props,
-        @Nullable IgniteUuid lockId, boolean cpProps, long modificationTime, boolean evictExclude) {
-        this(isDir, id, blockSize, len, affKey, listing, props, null, lockId, cpProps, modificationTime,
+        @Nullable IgniteUuid lockId, boolean cpProps, long accessTime, long modificationTime, boolean evictExclude) {
+        this(isDir, id, blockSize, len, affKey, listing, props, null, lockId, cpProps, accessTime,
             modificationTime, evictExclude);
     }
 
@@ -295,7 +297,7 @@ public final class IgfsFileInfo implements Externalizable {
             fileMap = new IgfsFileMap();
 
         this.fileMap = fileMap;
-        this.accessTime = accessTime;
+        this.accessTime = accessTime == ACCESS_TIME_TAKE_MODIFICATION_TIME ? modificationTime : accessTime;
         this.modificationTime = modificationTime;
 
         // Always make a copy of passed properties collection to escape concurrent modifications.

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/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 38914ea..c1e47a8 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
@@ -576,12 +576,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 IgfsMode mode = resolveMode(path);
 
-                IgfsFileInfo info = resolveFileInfo(path, mode);
-
-                if (info == null)
-                    return null;
-
-                return new IgfsFileImpl(path, info, data.groupBlockSize());
+                return resolveFileInfo(path, mode);
             }
         });
     }
@@ -856,10 +851,9 @@ public final class IgfsImpl implements IgfsEx {
                     Collection<IgfsFile> children = secondaryFs.listFiles(path);
 
                     for (IgfsFile child : children) {
-                        IgfsFileInfo fsInfo = new IgfsFileInfo(
-                            child.blockSize(), child.length(), evictExclude(path, false), child.properties());
+                        IgfsFileImpl impl = new IgfsFileImpl(child, data.groupBlockSize());
 
-                        files.add(new IgfsFileImpl(child.path(), fsInfo, data.groupBlockSize()));
+                        files.add(impl);
                     }
                 }
 
@@ -1557,7 +1551,7 @@ public final class IgfsImpl implements IgfsEx {
      * @return File info or {@code null} in case file is not found.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsFileInfo resolveFileInfo(IgfsPath path, IgfsMode mode) throws IgniteCheckedException {
+    private IgfsFileImpl resolveFileInfo(IgfsPath path, IgfsMode mode) throws IgniteCheckedException {
         assert path != null;
         assert mode != null;
 
@@ -1577,9 +1571,7 @@ public final class IgfsImpl implements IgfsEx {
                     IgfsFile status = secondaryFs.info(path);
 
                     if (status != null)
-                        info = status.isDirectory() ? new IgfsFileInfo(true, status.properties()) :
-                            new IgfsFileInfo(status.blockSize(), status.length(), null, null, false,
-                            status.properties());
+                        return new IgfsFileImpl(status, data.groupBlockSize());
                 }
 
                 break;
@@ -1588,7 +1580,10 @@ public final class IgfsImpl implements IgfsEx {
                 assert false : "Unknown mode: " + mode;
         }
 
-        return info;
+        if (info == null)
+            return null;
+
+        return new IgfsFileImpl(path, info, data.groupBlockSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/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 ab73e7d..a149b31 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
@@ -1669,7 +1669,7 @@ public class IgfsMetaManager extends IgfsManager {
 
                 assert validTxState(false);
 
-                id2InfoPrj.invokeAsync(parentId, new UpdateListingEntry(fileId, fileName, lenDelta, 0,
+                id2InfoPrj.invokeAsync(parentId, new UpdateListingEntry(fileId, fileName, lenDelta, -1,
                     modificationTime));
             }
             finally {
@@ -1995,7 +1995,8 @@ public class IgfsMetaManager extends IgfsManager {
                                     "the secondary file system because the path points to a directory: " + path);
 
                             IgfsFileInfo newInfo = new IgfsFileInfo(status.blockSize(), status.length(), affKey,
-                                composeLockId(false), igfsCtx.igfs().evictExclude(path, false), status.properties());
+                                composeLockId(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);
@@ -2626,9 +2627,11 @@ public class IgfsMetaManager extends IgfsManager {
                 }
 
                 // Recreate the path locally.
-                IgfsFileInfo curInfo = status.isDirectory() ? new IgfsFileInfo(true, status.properties()) :
+                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());
+                        igfsCtx.igfs().evictExclude(curPath, false), status.properties(),
+                        status.accessTime(), status.modificationTime());
 
                 IgniteUuid oldId = putIfAbsentNonTx(parentInfo.id(), components.get(i), curInfo);
 
@@ -3390,8 +3393,10 @@ public class IgfsMetaManager extends IgfsManager {
                     b = new DirectoryChainBuilder(path, dirProps, fileProps) {
                         /** {@inheritDoc} */
                         @Override protected IgfsFileInfo buildLeaf() {
+                            long t = System.currentTimeMillis();
+
                             return new IgfsFileInfo(blockSize, 0L, affKey, composeLockId(false),
-                                 evictExclude, leafProps);
+                                 evictExclude, leafProps, t, t);
                         }
                     };
 
@@ -3485,8 +3490,10 @@ public class IgfsMetaManager extends IgfsManager {
                                         id2InfoPrj.invoke(lowermostExistingInfo.id(), new UpdatePath(path));
 
                                         // Make a new locked info:
+                                        long t = System.currentTimeMillis();
+
                                         final IgfsFileInfo newFileInfo = new IgfsFileInfo(cfg.getBlockSize(), 0L,
-                                            affKey, composeLockId(false), evictExclude, fileProps);
+                                            affKey, composeLockId(false), evictExclude, fileProps, t, t);
 
                                         assert newFileInfo.lockId() != null; // locked info should be created.
 
@@ -3657,7 +3664,9 @@ public class IgfsMetaManager extends IgfsManager {
          * Builds leaf.
          */
         protected IgfsFileInfo buildLeaf()  {
-            return new IgfsFileInfo(true, leafProps);
+            long t = System.currentTimeMillis();
+
+            return new IgfsFileInfo(true, leafProps, t, t);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
index 84462fd..8d5ae7a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDataManagerSelfTest.java
@@ -166,8 +166,10 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
 
+            long t = System.currentTimeMillis();
+
             IgfsFileInfo info = new IgfsFileInfo(200, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
-                    false, null);
+                    false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -249,8 +251,10 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
 
+            long t = System.currentTimeMillis();
+
             IgfsFileInfo info = new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
-                false, null);
+                false, null, t, t);
 
             assertNull(mgr.dataBlock(info, path, 0, null).get());
 
@@ -338,9 +342,11 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         for (int i = 0; i < 10; i++) {
             IgfsPath path = new IgfsPath();
 
+            long t = System.currentTimeMillis();
+
             IgfsFileInfo info =
                 new IgfsFileInfo(blockSize, 0L, null, IgfsMetaManager.DELETE_LOCK_ID,
-                    false, null);
+                    false, null, t, t);
 
             IgfsFileAffinityRange range = new IgfsFileAffinityRange();
 
@@ -407,8 +413,10 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
         final int blockSize = 10;
         final int grpSize = blockSize * DATA_BLOCK_GROUP_CNT;
 
+        long t = System.currentTimeMillis();
+
         //IgfsFileInfo info = new IgfsFileInfo(blockSize, 0);
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null);
+        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
 
         for (int pos = 0; pos < 5 * grpSize; pos++) {
             assertEquals("Expects no affinity for zero length.", Collections.<IgfsBlockLocation>emptyList(),
@@ -456,7 +464,9 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
     public void testAffinity2() throws Exception {
         int blockSize = BLOCK_SIZE;
 
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null);
+        long t = System.currentTimeMillis();
+
+        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
 
         Collection<IgfsBlockLocation> affinity = mgr.affinity(info, 0, info.length());
 
@@ -487,7 +497,9 @@ public class IgfsDataManagerSelfTest extends IgfsCommonAbstractTest {
     public void testAffinityFileMap() throws Exception {
         int blockSize = BLOCK_SIZE;
 
-        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null);
+        long t = System.currentTimeMillis();
+
+        IgfsFileInfo info = new IgfsFileInfo(blockSize, 1024 * 1024, null, null, false, null, t, t);
 
         IgniteUuid affKey = IgniteUuid.randomUuid();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
index 683054b..0731436 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsDualAbstractSelfTest.java
@@ -30,7 +30,9 @@ import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsInputStream;
 import org.apache.ignite.igfs.IgfsMode;
 import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.GridTestUtils;
 
@@ -1202,7 +1204,8 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
 
                 try {
                     in0.read(readBuf);
-                } finally {
+                }
+                finally {
                     U.closeQuiet(in0);
                 }
 
@@ -1606,4 +1609,73 @@ public abstract class IgfsDualAbstractSelfTest extends IgfsAbstractSelfTest {
             clear(igfs, igfsSecondary);
         }
     }
+
+    /**
+     * Checks file access & modification time equality in the file itself and in the same file found through
+     * the listing of its parent.
+     *
+     * @param fs The file system.
+     * @param p The file path.
+     *
+     * @return Tuple of access and modification times of the file.
+     */
+    private T2<Long, Long> checkParentListingTime(IgfsSecondaryFileSystem fs, IgfsPath p) {
+        IgfsFile f0 = fs.info(p);
+
+        T2<Long, Long> t0 = new T2<>(f0.accessTime(), f0.modificationTime());
+
+        // Root cannot be seen through the parent listing:
+        if (!p.isSame(p.root())) {
+
+            assertNotNull(f0);
+
+            Collection<IgfsFile> listing = fs.listFiles(p.parent());
+
+            IgfsFile f1 = null;
+
+            for (IgfsFile fi : listing) {
+                if (fi.path().isSame(p)) {
+                    f1 = fi;
+
+                    break;
+                }
+            }
+
+            assertNotNull(f1); // file should be found in parent listing.
+
+            T2<Long, Long> t1 = new T2<>(f1.accessTime(), f1.modificationTime());
+
+            assertEquals(t0, t1);
+        }
+
+        return t0;
+    }
+
+    /**
+     * Test for file modification time upwards propagation when files are
+     * created on the secondary file system and initially
+     * unknown on the primary file system.
+     *
+     * @throws Exception On error.
+     */
+    public void testAccessAndModificationTimeUpwardsPropagation() throws Exception {
+        create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE, FILE2));
+
+        T2<Long,Long> timesDir0 = checkParentListingTime(igfsSecondaryFileSystem, DIR);
+        T2<Long,Long> timesSubDir0 = checkParentListingTime(igfsSecondaryFileSystem, SUBDIR);
+        T2<Long,Long> timesFile0 = checkParentListingTime(igfsSecondaryFileSystem, FILE);
+        T2<Long,Long> timesFile20 = checkParentListingTime(igfsSecondaryFileSystem, FILE2);
+
+        Thread.sleep(500L);
+
+        T2<Long,Long> timesDir1 = checkParentListingTime(igfs.asSecondary(), DIR);
+        T2<Long,Long> timesSubDir1 = checkParentListingTime(igfs.asSecondary(), SUBDIR);
+        T2<Long,Long> timesFile1 = checkParentListingTime(igfs.asSecondary(), FILE);
+        T2<Long,Long> timesFile21 = checkParentListingTime(igfs.asSecondary(), FILE2);
+
+        assertEquals(timesDir0, timesDir1);
+        assertEquals(timesSubDir0, timesSubDir1);
+        assertEquals(timesFile0, timesFile1);
+        assertEquals(timesFile20, timesFile21);
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
index 90ef404..feccdb5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfoSelfTest.java
@@ -52,10 +52,15 @@ public class IgfsFileInfoSelfTest extends IgfsCommonAbstractTest {
                 for (int i = 0; i < 10000; i++) {
                     testSerialization(new IgfsFileInfo());
                     testSerialization(new IgfsFileInfo());
-                    testSerialization(new IgfsFileInfo(true, null));
-                    testSerialization(new IgfsFileInfo(false, null));
 
-                    IgfsFileInfo rndInfo = new IgfsFileInfo(rnd.nextInt(max), null, false, null);
+                    long a = System.currentTimeMillis();
+                    long m = a + 1;
+
+                    testSerialization(new IgfsFileInfo(true, null, a, m));
+                    testSerialization(new IgfsFileInfo(false, null, a, m));
+
+                    IgfsFileInfo rndInfo =
+                        new IgfsFileInfo(rnd.nextInt(max), 0, false, null, a, m);
 
                     testSerialization(rndInfo);
                     testSerialization(new IgfsFileInfo(rndInfo, rnd.nextInt(max)));

http://git-wip-us.apache.org/repos/asf/ignite/blob/52e178f6/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index 12cd2ac..672b912 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -351,9 +351,10 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
             Collection<IgfsFile> res = new ArrayList<>(statuses.length);
 
             for (FileStatus status : statuses) {
-                IgfsFileInfo fsInfo = status.isDirectory() ? new IgfsFileInfo(true, properties(status)) :
-                    new IgfsFileInfo((int)status.getBlockSize(), status.getLen(), null, null, false,
-                    properties(status));
+                IgfsFileInfo fsInfo = status.isDirectory() ?
+                    new IgfsFileInfo(true, properties(status), status.getAccessTime(), status.getModificationTime()) :
+                    new IgfsFileInfo((int)status.getBlockSize(), status.getLen(), null, null, false, properties(status),
+                        status.getAccessTime(), status.getModificationTime());
 
                 res.add(new IgfsFileImpl(new IgfsPath(path, status.getPath().getName()), fsInfo, 1));
             }