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/06/10 12:48:36 UTC

[1/2] ignite git commit: IGNITE-3294: Getting closer to finish.

Repository: ignite
Updated Branches:
  refs/heads/ignite-3294 dfaefc4b7 -> 65ecd9d31


IGNITE-3294: Getting closer to finish.


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

Branch: refs/heads/ignite-3294
Commit: df7530826fe160a7cf3145c5d289bc4a977a162e
Parents: dfaefc4
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 10 15:38:53 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 10 15:38:53 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/igfs/IgfsImpl.java      |  4 +-
 .../processors/igfs/IgfsMetaManager.java        | 49 +++++++++++++-------
 .../processors/igfs/IgfsPathsCreateResult.java  | 16 ++++++-
 .../IgfsSecondaryFileSystemCreateContext.java   | 31 +++++++++----
 .../meta/IgfsMetaDirectoryCreateProcessor.java  | 40 ++++++++++------
 .../igfs/meta/IgfsMetaFileCreateProcessor.java  | 18 +++----
 6 files changed, 104 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/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 9f83f36..7234e52 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
@@ -1036,8 +1036,8 @@ public final class IgfsImpl implements IgfsEx {
                 IgfsSecondaryFileSystemCreateContext secondaryCtx = null;
 
                 if (mode != PRIMARY)
-                    secondaryCtx = new IgfsSecondaryFileSystemCreateContext(secondaryFs, simpleCreate,
-                        (short)replication, groupBlockSize(), bufSize);
+                    secondaryCtx = new IgfsSecondaryFileSystemCreateContext(secondaryFs, path, overwrite, simpleCreate,
+                        fileProps, (short)replication, groupBlockSize(), bufSize);
 
                 // Await for async ops completion if in DUAL mode.
                 if (mode != PRIMARY)

http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/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 5bc314d..30b9a82 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
@@ -2789,8 +2789,8 @@ public class IgfsMetaManager extends IgfsManager {
                         }
                         else {
                             // Create file and parent folders.
-                            IgfsPathsCreateResult res =
-                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+                            IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
+                                affKey, evictExclude, null);
 
                             if (res == null)
                                 continue;
@@ -2868,8 +2868,6 @@ public class IgfsMetaManager extends IgfsManager {
                     }
 
                     // Start TX.
-                    OutputStream secondaryOut = null;
-
                     try (IgniteInternalTx tx = startTx()) {
                         Map<IgniteUuid, IgfsEntryInfo> lockInfos = lockIds(lockIds);
 
@@ -2919,10 +2917,12 @@ public class IgfsMetaManager extends IgfsManager {
                             long newLen;
                             int newBlockSize;
 
+                            OutputStream secondaryOut = null;
+
                             if (secondaryCtx != null) {
-                                secondaryOut = secondaryCtx.create(path, overwrite, fileProps);
+                                secondaryOut = secondaryCtx.create();
 
-                                IgfsFile secondaryFile = secondaryCtx.info(path);
+                                IgfsFile secondaryFile = secondaryCtx.info();
 
                                 if (secondaryFile == null)
                                     throw fsException("Failed to open output stream to the file created in " +
@@ -2957,10 +2957,9 @@ public class IgfsMetaManager extends IgfsManager {
                             return new IgfsCreateResult(newInfo, secondaryOut);
                         }
                         else {
-                            // TODO: Handle this part.
                             // Create file and parent folders.
-                            IgfsPathsCreateResult res =
-                                createFile(pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+                            IgfsPathsCreateResult res = createFile(pathIds, lockInfos, dirProps, fileProps, blockSize,
+                                affKey, evictExclude, secondaryCtx);
 
                             if (res == null)
                                 continue;
@@ -2972,7 +2971,7 @@ public class IgfsMetaManager extends IgfsManager {
                             generateCreateEvents(res.createdPaths(), true);
 
                             // TODO: Set correct output stream.
-                            return new IgfsCreateResult(res.info(), null);
+                            return new IgfsCreateResult(res.info(), res.secondaryOutputStream());
                         }
                     }
                 }
@@ -3001,7 +3000,7 @@ public class IgfsMetaManager extends IgfsManager {
             throw new IgfsParentNotDirectoryException("Failed to create directory (parent " +
                 "element is not a directory)");
 
-        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false);
+        return createFileOrDirectory(true, pathIds, lockInfos, dirProps, null, 0, null, false, null);
     }
 
     /**
@@ -3014,22 +3013,25 @@ public class IgfsMetaManager extends IgfsManager {
      * @param blockSize Block size.
      * @param affKey Affinity key (optional)
      * @param evictExclude Evict exclude flag.
+     * @param secondaryCtx Secondary file system create context.
      * @return Result or {@code} if the first parent already contained child with the same name.
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private IgfsPathsCreateResult createFile(IgfsPathIds pathIds, Map<IgniteUuid, IgfsEntryInfo> lockInfos,
         Map<String, String> dirProps, Map<String, String> fileProps, int blockSize, @Nullable IgniteUuid affKey,
-        boolean evictExclude) throws IgniteCheckedException{
+        boolean evictExclude, @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx)
+        throws IgniteCheckedException{
         // Check if entry we are going to write to is directory.
         if (lockInfos.get(pathIds.lastExistingId()).isFile())
             throw new IgfsParentNotDirectoryException("Failed to open file for write " +
                 "(parent element is not a directory): " + pathIds.path());
 
-        return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude);
+        return createFileOrDirectory(false, pathIds, lockInfos, dirProps, fileProps, blockSize, affKey, evictExclude,
+            secondaryCtx);
     }
 
     /**
-     * Ceate file or directory.
+     * Create file or directory.
      *
      * @param dir Directory flag.
      * @param pathIds Path IDs.
@@ -3039,12 +3041,14 @@ public class IgfsMetaManager extends IgfsManager {
      * @param blockSize Block size.
      * @param affKey Affinity key.
      * @param evictExclude Evict exclude flag.
+     * @param secondaryCtx Secondary file system create context.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
     private IgfsPathsCreateResult createFileOrDirectory(boolean dir, IgfsPathIds pathIds,
         Map<IgniteUuid, IgfsEntryInfo> lockInfos, Map<String, String> dirProps, Map<String, String> fileProps,
-        int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude) throws IgniteCheckedException {
+        int blockSize, @Nullable IgniteUuid affKey, boolean evictExclude,
+        @Nullable IgfsSecondaryFileSystemCreateContext secondaryCtx) throws IgniteCheckedException {
         // This is our starting point.
         int lastExistingIdx = pathIds.lastExistingIndex();
         IgfsEntryInfo lastExistingInfo = lockInfos.get(pathIds.lastExistingId());
@@ -3060,6 +3064,12 @@ public class IgfsMetaManager extends IgfsManager {
         if (lastExistingInfo.hasChild(curPart))
             return null;
 
+        // Invoke secondary file system if needed.
+        OutputStream secondaryOut = null;
+
+        if (secondaryCtx != null)
+            secondaryOut = secondaryCtx.create();
+
         // First step: add new entry to the last existing element.
         id2InfoPrj.invoke(lastExistingInfo.id(), new IgfsMetaDirectoryListingAddProcessor(curPart,
             new IgfsListingEntry(curId, dir || !pathIds.isLastIndex(curIdx))));
@@ -3078,7 +3088,8 @@ public class IgfsMetaManager extends IgfsManager {
             String nextPart = pathIds.part(nextIdx);
             IgniteUuid nextId = pathIds.surrogateId(nextIdx);
 
-            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps,
+            // TODO: Use secondary FS
+            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(createTime, createTime, dirProps,
                 nextPart, new IgfsListingEntry(nextId, dir || !pathIds.isLastIndex(nextIdx))));
 
             // Save event.
@@ -3097,14 +3108,16 @@ public class IgfsMetaManager extends IgfsManager {
         IgfsEntryInfo info;
 
         if (dir)
-            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(createTime, dirProps));
+            // TODO: Use secondary FS
+            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(createTime, createTime, dirProps));
         else
+            // TODO: Use secondary FS
             info = invokeAndGet(curId, new IgfsMetaFileCreateProcessor(createTime, createTime, fileProps,
                 blockSize, affKey, createFileLockId(false), evictExclude, 0L));
 
         createdPaths.add(pathIds.path());
 
-        return new IgfsPathsCreateResult(createdPaths, info);
+        return new IgfsPathsCreateResult(createdPaths, info, secondaryOut);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
index bd13555..5f53f37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsPathsCreateResult.java
@@ -19,7 +19,9 @@ package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.jetbrains.annotations.Nullable;
 
+import java.io.OutputStream;
 import java.util.List;
 
 /**
@@ -32,15 +34,20 @@ public class IgfsPathsCreateResult {
     /** Info of the last created file. */
     private final IgfsEntryInfo info;
 
+    /** Secondary output stream. */
+    private final OutputStream secondaryOut;
+
     /**
      * Constructor.
      *
      * @param paths Created paths.
      * @param info Info of the last created file.
+     * @param secondaryOut Secondary output stream.
      */
-    public IgfsPathsCreateResult(List<IgfsPath> paths, IgfsEntryInfo info) {
+    public IgfsPathsCreateResult(List<IgfsPath> paths, IgfsEntryInfo info, @Nullable OutputStream secondaryOut) {
         this.paths = paths;
         this.info = info;
+        this.secondaryOut = secondaryOut;
     }
 
     /**
@@ -57,6 +64,13 @@ public class IgfsPathsCreateResult {
         return info;
     }
 
+    /**
+     * @return Secondary output stream.
+     */
+    public OutputStream secondaryOutputStream() {
+        return secondaryOut;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgfsPathsCreateResult.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
index f0fef70..fb90042 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemCreateContext.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.igfs;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.jetbrains.annotations.Nullable;
 
 import java.io.OutputStream;
 import java.util.Map;
@@ -31,9 +32,18 @@ public class IgfsSecondaryFileSystemCreateContext {
     /** File system. */
     private final IgfsSecondaryFileSystem fs;
 
+    /** Path. */
+    private final IgfsPath path;
+
+    /** Overwrite flag. */
+    private final boolean overwrite;
+
     /** Simple create flag. */
     private final boolean simpleCreate;
 
+    /** Properties. */
+    private final Map<String, String> props;
+
     /** Replication. */
     private final short replication;
 
@@ -47,14 +57,21 @@ public class IgfsSecondaryFileSystemCreateContext {
      * Constructor.
      *
      * @param fs File system.
+     * @param path Path.
+     * @param overwrite Overwrite flag.
      * @param simpleCreate Simple create flag.
-     * @param blockSize Block size.
+     * @param props Properties.
      * @param replication Replication.
+     * @param blockSize Block size.
+     * @param bufSize Buffer size.
      */
-    public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, boolean simpleCreate, short replication,
-        long blockSize, int bufSize) {
+    public IgfsSecondaryFileSystemCreateContext(IgfsSecondaryFileSystem fs, IgfsPath path, boolean overwrite,
+        boolean simpleCreate, @Nullable Map<String, String> props, short replication, long blockSize, int bufSize) {
         this.fs = fs;
+        this.path = path;
+        this.overwrite = overwrite;
         this.simpleCreate = simpleCreate;
+        this.props = props;
         this.replication = replication;
         this.blockSize = blockSize;
         this.bufSize = bufSize;
@@ -63,12 +80,9 @@ public class IgfsSecondaryFileSystemCreateContext {
     /**
      * Create file in the secondary file system.
      *
-     * @param path Path.
-     * @param overwrite Overwrite flag.
-     * @param props Properties.
      * @return Output stream.
      */
-    public OutputStream create(IgfsPath path, boolean overwrite, Map<String, String> props) {
+    public OutputStream create() {
         return simpleCreate ? fs.create(path, overwrite) :
             fs.create(path, bufSize, overwrite, replication, blockSize, props);
     }
@@ -76,10 +90,9 @@ public class IgfsSecondaryFileSystemCreateContext {
     /**
      * Get file info.
      *
-     * @param path Path.
      * @return File.
      */
-    public IgfsFile info(IgfsPath path) {
+    public IgfsFile info() {
         return fs.info(path);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
index eee9300..b016633 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaDirectoryCreateProcessor.java
@@ -48,8 +48,11 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** Create time. */
-    private long createTime;
+    /** Access time. */
+    private long accessTime;
+
+    /** Modification time. */
+    private long modificationTime;
 
     /** Properties. */
     private Map<String, String> props;
@@ -70,24 +73,27 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Create time.
+     * @param modificationTime Modification time.
      * @param props Properties.
      */
-    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props) {
-        this(createTime, props, null, null);
+    public IgfsMetaDirectoryCreateProcessor(long accessTime, long modificationTime, Map<String, String> props) {
+        this(accessTime, modificationTime, props, null, null);
     }
 
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Create time.
+     * @param modificationTime Modification time.
      * @param props Properties.
      * @param childName Child name.
      * @param childEntry Child entry.
      */
-    public IgfsMetaDirectoryCreateProcessor(long createTime, Map<String, String> props, String childName,
-        IgfsListingEntry childEntry) {
-        this.createTime = createTime;
+    public IgfsMetaDirectoryCreateProcessor(long accessTime, long modificationTime, Map<String, String> props,
+        String childName, IgfsListingEntry childEntry) {
+        this.accessTime = accessTime;
+        this.modificationTime = modificationTime;
         this.props = props;
         this.childName = childName;
         this.childEntry = childEntry;
@@ -101,8 +107,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
             entry.getKey(),
             null,
             props,
-            createTime,
-            createTime
+            accessTime,
+            modificationTime
         );
 
         if (childName != null)
@@ -115,7 +121,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -127,7 +134,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 
@@ -141,7 +149,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter out = writer.rawWriter();
 
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
+        out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
 
@@ -155,7 +164,8 @@ public class IgfsMetaDirectoryCreateProcessor implements EntryProcessor<IgniteUu
     @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
         BinaryRawReader in = reader.rawReader();
 
-        createTime = in.readLong();
+        accessTime = in.readLong();
+        modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/df753082/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
index 6c2bfa5..a3e9d48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/meta/IgfsMetaFileCreateProcessor.java
@@ -49,7 +49,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     private static final long serialVersionUID = 0L;
 
     /** Create time. */
-    private long createTime;
+    private long accessTime;
 
     /** Modification time. */
     private long modificationTime;
@@ -82,7 +82,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     /**
      * Constructor.
      *
-     * @param createTime Create time.
+     * @param accessTime Access time.
      * @param modificationTime Modification time.
      * @param props Properties.
      * @param blockSize Block size.
@@ -91,9 +91,9 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
      * @param evictExclude Evict exclude flag.
      * @param len File length.
      */
-    public IgfsMetaFileCreateProcessor(long createTime, long modificationTime, Map<String, String> props,
+    public IgfsMetaFileCreateProcessor(long accessTime, long modificationTime, Map<String, String> props,
         int blockSize, @Nullable IgniteUuid affKey, IgniteUuid lockId, boolean evictExclude, long len) {
-        this.createTime = createTime;
+        this.accessTime = accessTime;
         this.modificationTime = modificationTime;
         this.props = props;
         this.blockSize = blockSize;
@@ -114,7 +114,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
             lockId,
             evictExclude,
             props,
-            createTime,
+            accessTime,
             modificationTime
         );
 
@@ -125,7 +125,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
         out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
@@ -140,7 +140,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        createTime = in.readLong();
+        accessTime = in.readLong();
         modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);
@@ -157,7 +157,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     @Override public void writeBinary(BinaryWriter writer) throws BinaryObjectException {
         BinaryRawWriter out = writer.rawWriter();
 
-        out.writeLong(createTime);
+        out.writeLong(accessTime);
         out.writeLong(modificationTime);
 
         IgfsUtils.writeProperties(out, props);
@@ -174,7 +174,7 @@ public class IgfsMetaFileCreateProcessor implements EntryProcessor<IgniteUuid, I
     @Override public void readBinary(BinaryReader reader) throws BinaryObjectException {
         BinaryRawReader in = reader.rawReader();
 
-        createTime = in.readLong();
+        accessTime = in.readLong();
         modificationTime = in.readLong();
 
         props = IgfsUtils.readProperties(in);


[2/2] ignite git commit: IGNITE-3294: Implemented, but it is still mocked.

Posted by vo...@apache.org.
IGNITE-3294: Implemented, but it is still mocked.


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

Branch: refs/heads/ignite-3294
Commit: 65ecd9d31f407c76caceb7061def2f80dffe7464
Parents: df75308
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 10 15:48:07 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 10 15:48:07 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsMetaManager.java        | 101 ++++++++++++++++---
 1 file changed, 89 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/65ecd9d3/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 30b9a82..37c3830 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
@@ -3080,21 +3080,44 @@ public class IgfsMetaManager extends IgfsManager {
         List<IgfsPath> createdPaths = new ArrayList<>(pathIds.count() - curIdx);
 
         // Second step: create middle directories.
-        long createTime = System.currentTimeMillis();
+        long curTime = System.currentTimeMillis();
 
         while (curIdx < pathIds.count() - 1) {
+            lastCreatedPath = new IgfsPath(lastCreatedPath, curPart);
+
             int nextIdx = curIdx + 1;
 
             String nextPart = pathIds.part(nextIdx);
             IgniteUuid nextId = pathIds.surrogateId(nextIdx);
 
-            // TODO: Use secondary FS
-            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(createTime, createTime, dirProps,
+            long accessTime;
+            long modificationTime;
+            Map<String, String> props;
+
+            if (secondaryCtx != null) {
+                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(lastCreatedPath);
+
+                if (secondaryInfo == null)
+                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
+                        "modified concurrnetly: " + lastCreatedPath);
+                else if (secondaryInfo.isFile())
+                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
+                        "not directory: " + lastCreatedPath);
+
+                accessTime = secondaryInfo.accessTime();
+                modificationTime = secondaryInfo.modificationTime();
+                props = secondaryInfo.properties();
+            }
+            else {
+                accessTime = curTime;
+                modificationTime = curTime;
+                props = dirProps;
+            }
+
+            id2InfoPrj.invoke(curId, new IgfsMetaDirectoryCreateProcessor(accessTime, modificationTime, props,
                 nextPart, new IgfsListingEntry(nextId, dir || !pathIds.isLastIndex(nextIdx))));
 
             // Save event.
-            lastCreatedPath = new IgfsPath(lastCreatedPath, curPart);
-
             createdPaths.add(lastCreatedPath);
 
             // Advance things further.
@@ -3107,13 +3130,67 @@ public class IgfsMetaManager extends IgfsManager {
         // Third step: create leaf.
         IgfsEntryInfo info;
 
-        if (dir)
-            // TODO: Use secondary FS
-            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(createTime, createTime, dirProps));
-        else
-            // TODO: Use secondary FS
-            info = invokeAndGet(curId, new IgfsMetaFileCreateProcessor(createTime, createTime, fileProps,
-                blockSize, affKey, createFileLockId(false), evictExclude, 0L));
+        if (dir) {
+            long accessTime;
+            long modificationTime;
+            Map<String, String> props;
+
+            if (secondaryCtx != null) {
+                IgfsFile secondaryInfo = secondaryCtx.fileSystem().info(pathIds.path());
+
+                if (secondaryInfo == null)
+                    throw new IgfsException("Failed to perform operation because secondary file system path was " +
+                        "modified concurrnetly: " + pathIds.path());
+                else if (secondaryInfo.isFile())
+                    throw new IgfsException("Failed to perform operation because secondary file system entity is " +
+                        "not directory: " + lastCreatedPath);
+
+                accessTime = secondaryInfo.accessTime();
+                modificationTime = secondaryInfo.modificationTime();
+                props = secondaryInfo.properties();
+            }
+            else {
+                accessTime = curTime;
+                modificationTime = curTime;
+                props = dirProps;
+            }
+
+            info = invokeAndGet(curId, new IgfsMetaDirectoryCreateProcessor(accessTime, modificationTime, props));
+        }
+        else {
+            long newAccessTime;
+            long newModificationTime;
+            Map<String, String> newProps;
+            long newLen;
+            int newBlockSize;
+
+            if (secondaryCtx != null) {
+                IgfsFile secondaryFile = secondaryCtx.info();
+
+                if (secondaryFile == null)
+                    throw fsException("Failed to open output stream to the file created in " +
+                        "the secondary file system because it no longer exists: " + pathIds.path());
+                else if (secondaryFile.isDirectory())
+                    throw fsException("Failed to open output stream to the file created in " +
+                        "the secondary file system because the path points to a directory: " + pathIds.path());
+
+                newAccessTime = secondaryFile.accessTime();
+                newModificationTime = secondaryFile.modificationTime();
+                newProps = secondaryFile.properties();
+                newLen = secondaryFile.length();
+                newBlockSize = secondaryFile.blockSize();
+            }
+            else {
+                newAccessTime = curTime;
+                newModificationTime = curTime;
+                newProps = fileProps;
+                newLen = 0L;
+                newBlockSize = blockSize;
+            }
+
+            info = invokeAndGet(curId, new IgfsMetaFileCreateProcessor(newAccessTime, newModificationTime, newProps,
+                newBlockSize, affKey, createFileLockId(false), evictExclude, newLen));
+        }
 
         createdPaths.add(pathIds.path());