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/08/15 07:38:57 UTC

[28/30] ignite git commit: Moved al changes to proper package.

Moved al changes to proper package.


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

Branch: refs/heads/ignite-1926
Commit: e0771b88326c8bef2e58d9a8a59d4f4851817b37
Parents: 72a7604
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Aug 15 10:36:00 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Aug 15 10:36:00 2016 +0300

----------------------------------------------------------------------
 .../local/LocalIgfsSecondaryFileSystem.java     | 396 +++++++++++++++++++
 .../igfs/secondary/local/package-info.java      |  22 ++
 .../ignite/igfs/secondary/package-info.java     |   2 +-
 .../local/LocalFileSystemIgfsFile.java          | 134 +++++++
 ...fsSecondaryFileSystemPositionedReadable.java |  65 +++
 ...SecondaryFileSystemDualAbstractSelfTest.java |  76 ++++
 ...ondaryFileSystemDualAsyncClientSelfTest.java |  28 ++
 ...calSecondaryFileSystemDualAsyncSelfTest.java |  32 ++
 ...condaryFileSystemDualSyncClientSelfTest.java |  28 ++
 ...ocalSecondaryFileSystemDualSyncSelfTest.java |  32 ++
 ...IgfsLocalSecondaryFileSystemTestAdapter.java | 141 +++++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   9 +
 .../hadoop/fs/LocalFileSystemIgfsFile.java      | 134 -------
 .../hadoop/fs/LocalIgfsSecondaryFileSystem.java | 394 ------------------
 ...fsSecondaryFileSystemPositionedReadable.java |  65 ---
 ...LocalIgfsSecondaryFileSystemTestAdapter.java | 146 -------
 ...SecondaryFileSystemDualAbstractSelfTest.java |  76 ----
 ...ondaryFileSystemDualAsyncClientSelfTest.java |  28 --
 ...calSecondaryFileSystemDualAsyncSelfTest.java |  30 --
 ...condaryFileSystemDualSyncClientSelfTest.java |  28 --
 ...ocalSecondaryFileSystemDualSyncSelfTest.java |  30 --
 21 files changed, 964 insertions(+), 932 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
new file mode 100644
index 0000000..3d3a350
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/LocalIgfsSecondaryFileSystem.java
@@ -0,0 +1,396 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs.secondary.local;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.igfs.IgfsException;
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
+import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException;
+import org.apache.ignite.igfs.IgfsPathNotFoundException;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.processors.igfs.secondary.local.LocalFileSystemIgfsFile;
+import org.apache.ignite.internal.processors.igfs.secondary.local.LocalIgfsSecondaryFileSystemPositionedReadable;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lifecycle.LifecycleAware;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.file.Files;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Secondary file system which delegates to local file system.
+ */
+public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware {
+    /** Default buffer size. */
+    private static final int DFLT_BUF_SIZE = 8 * 1024;
+
+    /** Path that will be added to each passed path. */
+    private String workDir;
+
+    /**
+     * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception.
+     *
+     * @param e Exception to check.
+     * @param msg Detailed error message.
+     * @return Appropriate exception.
+     */
+    private IgfsException handleSecondaryFsError(IOException e, String msg) {
+        if (e instanceof FileNotFoundException)
+            return new IgfsPathNotFoundException(e);
+        else
+            return new IgfsException(msg, e);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean exists(IgfsPath path) {
+        return fileForPath(path).exists();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
+        throw new UnsupportedOperationException("Update operation is not yet supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rename(IgfsPath src, IgfsPath dest) {
+        File srcFile = fileForPath(src);
+        File destFile = fileForPath(dest);
+
+        if (!srcFile.exists())
+            throw new IgfsPathNotFoundException("Failed to perform rename because source path not found: " + src);
+
+        if (srcFile.isDirectory() && destFile.isFile())
+            throw new IgfsPathIsNotDirectoryException("Failed to perform rename because destination path is " +
+                "directory and source path is file [src=" + src + ", dest=" + dest + ']');
+
+        try {
+            if (destFile.isDirectory())
+                Files.move(srcFile.toPath(), destFile.toPath().resolve(srcFile.getName()));
+            else if(!srcFile.renameTo(destFile))
+                throw new IgfsException("Failed to perform rename (underlying file system returned false) " +
+                    "[src=" + src + ", dest=" + dest + ']');
+        }
+        catch (IOException e) {
+            throw handleSecondaryFsError(e, "Failed to rename [src=" + src + ", dest=" + dest + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public boolean delete(IgfsPath path, boolean recursive) {
+        File f = fileForPath(path);
+
+        if (!recursive || !f.isDirectory())
+            return f.delete();
+        else
+            return deleteDirectory(f);
+    }
+
+    /**
+     * Delete directory recursively.
+     *
+     * @param dir Directory.
+     * @return {@code true} if successful.
+     */
+    private boolean deleteDirectory(File dir) {
+        File[] entries = dir.listFiles();
+
+        if (entries != null) {
+            for (File entry : entries) {
+                if (entry.isDirectory())
+                    deleteDirectory(entry);
+                else if (entry.isFile()) {
+                    if (!entry.delete())
+                        return false;
+                }
+                else
+                    throw new UnsupportedOperationException("Symlink deletion is not yet supported: " + entry);
+            }
+        }
+
+        return dir.delete();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(IgfsPath path) {
+        if (!mkdirs0(fileForPath(path)))
+            throw new IgniteException("Failed to make directories (underlying file system returned false): " + path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
+        mkdirs(path);
+    }
+
+    /**
+     * Create directories.
+     *
+     * @param dir Directory.
+     * @return Result.
+     */
+    private boolean mkdirs0(@Nullable File dir) {
+        if (dir == null)
+            return true; // Nothing to create.
+
+        if (dir.exists())
+            // Already exists, so no-op.
+            return dir.isDirectory();
+        else {
+            File parentDir = dir.getParentFile();
+
+            if (!mkdirs0(parentDir)) // Create parent first.
+                return false;
+
+            boolean res = dir.mkdir();
+
+            if (!res)
+                res = dir.exists(); // Tolerate concurrent creation.
+
+            return res;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
+        File[] entries = listFiles0(path);
+
+        if (F.isEmpty(entries))
+            return Collections.emptySet();
+        else {
+            Collection<IgfsPath> res = U.newHashSet(entries.length);
+
+            for (File entry : entries)
+                res.add(igfsPath(entry));
+
+            return res;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
+        File[] entries = listFiles0(path);
+
+        if (F.isEmpty(entries))
+            return Collections.emptySet();
+        else {
+            Collection<IgfsFile> res = U.newHashSet(entries.length);
+
+            for (File entry : entries) {
+                IgfsFile info = info(igfsPath(entry));
+
+                if (info != null)
+                    res.add(info);
+            }
+
+            return res;
+        }
+    }
+
+    /**
+     * Returns an array of File object. Under the specific path.
+     *
+     * @param path IGFS path.
+     * @return Array of File objects.
+     */
+    @Nullable private File[] listFiles0(IgfsPath path) {
+        File f = fileForPath(path);
+
+        if (!f.exists())
+            throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
+        else
+            return f.listFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) {
+        try {
+            FileInputStream in = new FileInputStream(fileForPath(path));
+
+            return new LocalIgfsSecondaryFileSystemPositionedReadable(in, bufSize);
+        }
+        catch (IOException e) {
+            throw handleSecondaryFsError(e, "Failed to open file for read: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream create(IgfsPath path, boolean overwrite) {
+        return create0(path, overwrite, DFLT_BUF_SIZE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
+        long blockSize, @Nullable Map<String, String> props) {
+        return create0(path, overwrite, bufSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
+        @Nullable Map<String, String> props) {
+        try {
+            File file = fileForPath(path);
+
+            boolean exists = file.exists();
+
+            if (exists)
+                return new BufferedOutputStream(new FileOutputStream(file, true), bufSize);
+            else {
+                if (create)
+                    return create0(path, false, bufSize);
+                else
+                    throw new IgfsPathNotFoundException("Failed to append to file because it doesn't exist: " + path);
+            }
+        }
+        catch (IOException e) {
+            throw handleSecondaryFsError(e, "Failed to append to file because it doesn't exist: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(final IgfsPath path) {
+        File f = fileForPath(path);
+
+        if (!f.exists())
+            return null;
+
+        boolean isDir = f.isDirectory();
+
+        if (isDir)
+            return new LocalFileSystemIgfsFile(path, false, true, 0, f.lastModified(), 0, null);
+        else
+            return new LocalFileSystemIgfsFile(path, f.isFile(), false, 0, f.lastModified(), f.length(), null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long usedSpaceSize() {
+        throw new UnsupportedOperationException("usedSpaceSize operation is not yet supported.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (workDir != null)
+            workDir = new File(workDir).getAbsolutePath();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /**
+     * Get work directory.
+     *
+     * @return Work directory.
+     */
+    @Nullable public String getWorkDirectory() {
+        return workDir;
+    }
+
+    /**
+     * Set work directory.
+     *
+     * @param workDir Work directory.
+     */
+    public void setWorkDirectory(@Nullable String workDir) {
+        this.workDir = workDir;
+    }
+
+    /**
+     * Create file for IGFS path.
+     *
+     * @param path IGFS path.
+     * @return File object.
+     */
+    private File fileForPath(IgfsPath path) {
+        if (workDir == null)
+            return new File(path.toString());
+        else {
+            if ("/".equals(path.toString()))
+                return new File(workDir);
+            else
+                return new File(workDir, path.toString());
+        }
+    }
+
+    /**
+     * Create IGFS path for file.
+     *
+     * @param f File object.
+     * @return IFGS path.
+     * @throws IgfsException If failed.
+     */
+    private IgfsPath igfsPath(File f) throws IgfsException {
+        String path = f.getAbsolutePath();
+
+        if (workDir != null) {
+            if (!path.startsWith(workDir))
+                throw new IgfsException("Path is not located in the work directory [workDir=" + workDir +
+                    ", path=" + path + ']');
+
+            path = path.substring(workDir.length(), path.length());
+        }
+
+        return new IgfsPath(path);
+    }
+
+    /**
+     * Internal create routine.
+     *
+     * @param path Path.
+     * @param overwrite Overwirte flag.
+     * @param bufSize Buffer size.
+     * @return Output stream.
+     */
+    private OutputStream create0(IgfsPath path, boolean overwrite, int bufSize) {
+        File file = fileForPath(path);
+
+        boolean exists = file.exists();
+
+        if (exists) {
+            if (!overwrite)
+                throw new IgfsPathAlreadyExistsException("Failed to create a file because it already exists: " + path);
+        }
+        else {
+            File parent = file.getParentFile();
+
+            if (!mkdirs0(parent))
+                throw new IgfsException("Failed to create parent directory for file (underlying file system " +
+                    "returned false): " + path);
+        }
+
+        try {
+            return new BufferedOutputStream(new FileOutputStream(file), bufSize);
+        }
+        catch (IOException e) {
+            throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + ']');
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java
new file mode 100644
index 0000000..80bdce2
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/local/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains APIs for IGFS secondary file system.
+ */
+package org.apache.ignite.igfs.secondary.local;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java
index 4914c47..471651f 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/package-info.java
@@ -17,6 +17,6 @@
 
 /**
  * <!-- Package description. -->
- * Contains APIs for IGFS secondary file system.
+ * Contains APIs for IGFS secondary file system base on local file system.
  */
 package org.apache.ignite.igfs.secondary;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java
new file mode 100644
index 0000000..5abe4eb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalFileSystemIgfsFile.java
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs.secondary.local;
+
+import org.apache.ignite.igfs.IgfsFile;
+import org.apache.ignite.igfs.IgfsPath;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Implementation of the IgfsFile interface for the local filesystem.
+ */
+public class LocalFileSystemIgfsFile implements IgfsFile {
+    /** Path. */
+    private final IgfsPath path;
+
+    /** Flags. */
+    private final byte flags;
+
+    /** Block size. */
+    private final int blockSize;
+
+    /** Modification time. */
+    private final long modTime;
+
+    /** Length. */
+    private final long len;
+
+    /** Properties. */
+    private final Map<String, String> props;
+
+    /**
+     * @param path IGFS path.
+     * @param isFile Path is a file.
+     * @param isDir Path is a directory.
+     * @param blockSize Block size in bytes.
+     * @param modTime Modification time in millis.
+     * @param len File length in bytes.
+     * @param props Properties.
+     */
+    public LocalFileSystemIgfsFile(IgfsPath path, boolean isFile, boolean isDir, int blockSize,
+        long modTime, long len, Map<String, String> props) {
+
+        assert !isDir || blockSize == 0 : "blockSize must be 0 for dirs. [blockSize=" + blockSize + ']';
+        assert !isDir || len == 0 : "length must be 0 for dirs. [length=" + len + ']';
+
+        this.path = path;
+        this.flags = IgfsUtils.flags(isDir, isFile);
+        this.blockSize = blockSize;
+        this.modTime = modTime;
+        this.len = len;
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsPath path() {
+        return path;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isFile() {
+        return IgfsUtils.isFile(flags);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDirectory() {
+        return IgfsUtils.isDirectory(flags);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int blockSize() {
+        return blockSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long groupBlockSize() {
+        return blockSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long accessTime() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long modificationTime() {
+        return modTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String property(String name) throws IllegalArgumentException {
+        return property(name, null);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String property(String name, @Nullable String dfltVal) {
+        if (props != null) {
+            String res = props.get(name);
+
+            if (res != null)
+                return res;
+        }
+
+        return dfltVal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, String> properties() {
+        return props != null ? props : Collections.<String, String>emptyMap();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long length() {
+        return len;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java
new file mode 100644
index 0000000..ebf56ad
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/secondary/local/LocalIgfsSecondaryFileSystemPositionedReadable.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs.secondary.local;
+
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
+
+import java.io.BufferedInputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+
+/**
+ * Positioned readable interface for local secondary file system.
+ */
+public class LocalIgfsSecondaryFileSystemPositionedReadable extends BufferedInputStream
+    implements IgfsSecondaryFileSystemPositionedReadable {
+    /** Last read position. */
+    private long lastReadPos;
+
+    /**
+     * Constructor.
+     *
+     * @param in Input stream.
+     * @param bufSize Buffer size.
+     */
+    public LocalIgfsSecondaryFileSystemPositionedReadable(FileInputStream in, int bufSize) {
+        super(in, bufSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(long readPos, byte[] buf, int off, int len) throws IOException {
+        if (in == null)
+            throw new IOException("Stream is closed.");
+
+        if (readPos < lastReadPos || readPos + len > lastReadPos + this.buf.length) {
+            ((FileInputStream)in).getChannel().position(readPos);
+
+            pos = 0;
+            count = 0;
+        }
+
+        int bytesRead = read(buf, off, len);
+
+        if (bytesRead != -1) {
+            // Advance last read position only if we really read some bytes from the stream.
+            lastReadPos = readPos + bytesRead;
+        }
+
+        return bytesRead;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
new file mode 100644
index 0000000..c2f5633
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAbstractSelfTest.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.igfs.IgfsMode;
+import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
+import org.apache.ignite.igfs.secondary.local.LocalIgfsSecondaryFileSystem;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+import java.io.File;
+
+/**
+ * Abstract test for Hadoop 1.0 file system stack.
+ */
+public abstract class IgfsLocalSecondaryFileSystemDualAbstractSelfTest extends IgfsDualAbstractSelfTest {
+    /** */
+    private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work"
+        + File.separatorChar + "fs";
+
+    /** Constructor.
+     * @param mode IGFS mode.
+     */
+    public IgfsLocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) {
+        super(mode);
+    }
+
+    /**
+     * Creates secondary filesystems.
+     * @return IgfsSecondaryFileSystem
+     * @throws Exception On failure.
+     */
+    @Override protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
+       final File workDir = new File(FS_WORK_DIR);
+
+        if (!workDir.exists())
+            assert workDir.mkdirs();
+
+        LocalIgfsSecondaryFileSystem second = new LocalIgfsSecondaryFileSystem();
+
+        second.setWorkDirectory(workDir.getAbsolutePath());
+
+        igfsSecondary = new IgfsLocalSecondaryFileSystemTestAdapter(workDir);
+
+        return second;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean permissionsSupported() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean propertiesSupported() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean timesSupported() {
+        return false;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java
new file mode 100644
index 0000000..fd9e10c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * DUAL_ASYNC mode.
+ */
+public class IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest extends IgfsLocalSecondaryFileSystemDualAsyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java
new file mode 100644
index 0000000..65a2064
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualAsyncSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.igfs.IgfsMode;
+
+/**
+ * DUAL_ASYNC mode test.
+ */
+public class IgfsLocalSecondaryFileSystemDualAsyncSelfTest extends IgfsLocalSecondaryFileSystemDualAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsLocalSecondaryFileSystemDualAsyncSelfTest() {
+        super(IgfsMode.DUAL_ASYNC);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java
new file mode 100644
index 0000000..9358850
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+/**
+ * DUAL_SYNC mode.
+ */
+public class IgfsLocalSecondaryFileSystemDualSyncClientSelfTest extends IgfsLocalSecondaryFileSystemDualSyncSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean client() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java
new file mode 100644
index 0000000..fef4f7c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemDualSyncSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.igfs.IgfsMode;
+
+/**
+ * DUAL_SYNC mode.
+ */
+public class IgfsLocalSecondaryFileSystemDualSyncSelfTest extends IgfsLocalSecondaryFileSystemDualAbstractSelfTest {
+    /**
+     * Constructor.
+     */
+    public IgfsLocalSecondaryFileSystemDualSyncSelfTest() {
+        super(IgfsMode.DUAL_SYNC);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java
new file mode 100644
index 0000000..12714c4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsLocalSecondaryFileSystemTestAdapter.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.igfs;
+
+import org.apache.ignite.internal.util.typedef.T2;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.nio.file.DirectoryStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.nio.file.StandardOpenOption;
+import java.util.Map;
+
+/**
+ * Adapter for local secondary file system.
+ */
+public class IgfsLocalSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter {
+    /** */
+    private final String workDir;
+
+    /**
+     * @param workDir Work dir.
+     */
+    public IgfsLocalSecondaryFileSystemTestAdapter(final File workDir) {
+        this.workDir = workDir.getAbsolutePath();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() throws IOException {
+        return "local";
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean exists(final String path) throws IOException {
+        return Files.exists(path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(final String path, final boolean recursive) throws IOException {
+        if (recursive)
+            return deleteRecursively(path(path));
+        else
+            return path(path).toFile().delete();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(final String path) throws IOException {
+        Files.createDirectory(path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void format() throws IOException {
+        try (DirectoryStream<Path> stream = Files.newDirectoryStream(Paths.get(workDir))) {
+            for (Path innerPath : stream)
+                deleteRecursively(innerPath);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, String> properties(final String path) throws IOException {
+        throw new UnsupportedOperationException("properties");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String permissions(String path) throws IOException {
+        throw new UnsupportedOperationException("permissions");
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputStream openInputStream(final String path) throws IOException {
+        return Files.newInputStream(path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream openOutputStream(final String path, final boolean append) throws IOException {
+        if (append)
+            return Files.newOutputStream(path(path), StandardOpenOption.APPEND);
+        else
+            return Files.newOutputStream(path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public T2<Long, Long> times(String path) throws IOException {
+        throw new UnsupportedOperationException("times");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEx igfs() {
+        return null;
+    }
+
+    /**
+     * Convert path.
+     *
+     * @param path String path.
+     * @return Java File API path.
+     */
+    private Path path(String path) {
+        return Paths.get(workDir + path);
+    }
+
+    /**
+     * Delete recursively.
+     *
+     * @param path Path.
+     * @throws IOException If failed.
+     */
+    private boolean deleteRecursively(Path path) throws IOException {
+        if (Files.isDirectory(path)) {
+            try (DirectoryStream<Path> stream = Files.newDirectoryStream(path)) {
+                for (Path innerPath : stream) {
+                    boolean res = deleteRecursively(innerPath);
+
+                    if (!res)
+                        return false;
+                }
+            }
+        }
+
+        return path.toFile().delete();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index dc1379f..44199d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -56,6 +56,10 @@ import org.apache.ignite.internal.processors.igfs.IgfsSizeSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsStartCacheTest;
 import org.apache.ignite.internal.processors.igfs.IgfsStreamsSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsTaskSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualAsyncSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualSyncClientSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsLocalSecondaryFileSystemDualSyncSelfTest;
 import org.apache.ignite.internal.processors.igfs.split.IgfsByteDelimiterRecordResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.split.IgfsFixedLengthRecordResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.split.IgfsNewLineDelimiterRecordResolverSelfTest;
@@ -88,6 +92,11 @@ public class IgniteIgfsTestSuite extends TestSuite {
         suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
         suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
 
+        suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualSyncSelfTest.class));
+        suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualAsyncSelfTest.class));
+        suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualSyncClientSelfTest.class));
+        suite.addTest(new TestSuite(IgfsLocalSecondaryFileSystemDualAsyncClientSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsSizeSelfTest.class));
         suite.addTest(new TestSuite(IgfsAttributesSelfTest.class));
         suite.addTest(new TestSuite(IgfsFileInfoSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalFileSystemIgfsFile.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalFileSystemIgfsFile.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalFileSystemIgfsFile.java
deleted file mode 100644
index 2ecbbf1..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalFileSystemIgfsFile.java
+++ /dev/null
@@ -1,134 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.hadoop.fs;
-
-import java.util.Collections;
-import java.util.Map;
-
-import org.apache.ignite.igfs.IgfsFile;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.processors.igfs.IgfsUtils;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Implementation of the IgfsFile interface for the local filesystem.
- */
-class LocalFileSystemIgfsFile implements IgfsFile {
-    /** Path. */
-    private final IgfsPath path;
-
-    /** Flags. */
-    private final byte flags;
-
-    /** Block size. */
-    private final int blockSize;
-
-    /** Modification time. */
-    private final long modTime;
-
-    /** Length. */
-    private final long len;
-
-    /** Properties. */
-    private final Map<String, String> props;
-
-    /**
-     * @param path IGFS path.
-     * @param isFile Path is a file.
-     * @param isDir Path is a directory.
-     * @param blockSize Block size in bytes.
-     * @param modTime Modification time in millis.
-     * @param len File length in bytes.
-     * @param props Properties.
-     */
-    LocalFileSystemIgfsFile(IgfsPath path, boolean isFile, boolean isDir, int blockSize,
-        long modTime, long len, Map<String, String> props) {
-
-        assert !isDir || blockSize == 0 : "blockSize must be 0 for dirs. [blockSize=" + blockSize + ']';
-        assert !isDir || len == 0 : "length must be 0 for dirs. [length=" + len + ']';
-
-        this.path = path;
-        this.flags = IgfsUtils.flags(isDir, isFile);
-        this.blockSize = blockSize;
-        this.modTime = modTime;
-        this.len = len;
-        this.props = props;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsPath path() {
-        return path;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isFile() {
-        return IgfsUtils.isFile(flags);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isDirectory() {
-        return IgfsUtils.isDirectory(flags);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int blockSize() {
-        return blockSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long groupBlockSize() {
-        return blockSize();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long accessTime() {
-        return 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long modificationTime() {
-        return modTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String property(String name) throws IllegalArgumentException {
-        return property(name, null);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String property(String name, @Nullable String dfltVal) {
-        if (props != null) {
-            String res = props.get(name);
-
-            if (res != null)
-                return res;
-        }
-
-        return dfltVal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, String> properties() {
-        return props != null ? props : Collections.<String, String>emptyMap();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long length() {
-        return len;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystem.java
deleted file mode 100644
index 850e927..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystem.java
+++ /dev/null
@@ -1,394 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.hadoop.fs;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.igfs.IgfsException;
-import org.apache.ignite.igfs.IgfsFile;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.igfs.IgfsPathAlreadyExistsException;
-import org.apache.ignite.igfs.IgfsPathIsNotDirectoryException;
-import org.apache.ignite.igfs.IgfsPathNotFoundException;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lifecycle.LifecycleAware;
-import org.jetbrains.annotations.Nullable;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.nio.file.Files;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Map;
-
-/**
- * Secondary file system which delegates to local file system.
- */
-public class LocalIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, LifecycleAware {
-    /** Default buffer size. */
-    private static final int DFLT_BUF_SIZE = 8 * 1024;
-
-    /** Path that will be added to each passed path. */
-    private String workDir;
-
-    /**
-     * Heuristically checks if exception was caused by invalid HDFS version and returns appropriate exception.
-     *
-     * @param e Exception to check.
-     * @param msg Detailed error message.
-     * @return Appropriate exception.
-     */
-    private IgfsException handleSecondaryFsError(IOException e, String msg) {
-        if (e instanceof FileNotFoundException)
-            return new IgfsPathNotFoundException(e);
-        else
-            return new IgfsException(msg, e);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean exists(IgfsPath path) {
-        return fileForPath(path).exists();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
-        throw new UnsupportedOperationException("Update operation is not yet supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void rename(IgfsPath src, IgfsPath dest) {
-        File srcFile = fileForPath(src);
-        File destFile = fileForPath(dest);
-
-        if (!srcFile.exists())
-            throw new IgfsPathNotFoundException("Failed to perform rename because source path not found: " + src);
-
-        if (srcFile.isDirectory() && destFile.isFile())
-            throw new IgfsPathIsNotDirectoryException("Failed to perform rename because destination path is " +
-                "directory and source path is file [src=" + src + ", dest=" + dest + ']');
-
-        try {
-            if (destFile.isDirectory())
-                Files.move(srcFile.toPath(), destFile.toPath().resolve(srcFile.getName()));
-            else if(!srcFile.renameTo(destFile))
-                throw new IgfsException("Failed to perform rename (underlying file system returned false) " +
-                    "[src=" + src + ", dest=" + dest + ']');
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to rename [src=" + src + ", dest=" + dest + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("ConstantConditions")
-    @Override public boolean delete(IgfsPath path, boolean recursive) {
-        File f = fileForPath(path);
-
-        if (!recursive || !f.isDirectory())
-            return f.delete();
-        else
-            return deleteDirectory(f);
-    }
-
-    /**
-     * Delete directory recursively.
-     *
-     * @param dir Directory.
-     * @return {@code true} if successful.
-     */
-    private boolean deleteDirectory(File dir) {
-        File[] entries = dir.listFiles();
-
-        if (entries != null) {
-            for (File entry : entries) {
-                if (entry.isDirectory())
-                    deleteDirectory(entry);
-                else if (entry.isFile()) {
-                    if (!entry.delete())
-                        return false;
-                }
-                else
-                    throw new UnsupportedOperationException("Symlink deletion is not yet supported: " + entry);
-            }
-        }
-
-        return dir.delete();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(IgfsPath path) {
-        if (!mkdirs0(fileForPath(path)))
-            throw new IgniteException("Failed to make directories (underlying file system returned false): " + path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
-        mkdirs(path);
-    }
-
-    /**
-     * Create directories.
-     *
-     * @param dir Directory.
-     * @return Result.
-     */
-    private boolean mkdirs0(@Nullable File dir) {
-        if (dir == null)
-            return true; // Nothing to create.
-
-        if (dir.exists())
-            // Already exists, so no-op.
-            return dir.isDirectory();
-        else {
-            File parentDir = dir.getParentFile();
-
-            if (!mkdirs0(parentDir)) // Create parent first.
-                return false;
-
-            boolean res = dir.mkdir();
-
-            if (!res)
-                res = dir.exists(); // Tolerate concurrent creation.
-
-            return res;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
-        File[] entries = listFiles0(path);
-
-        if (F.isEmpty(entries))
-            return Collections.emptySet();
-        else {
-            Collection<IgfsPath> res = U.newHashSet(entries.length);
-
-            for (File entry : entries)
-                res.add(igfsPath(entry));
-
-            return res;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
-        File[] entries = listFiles0(path);
-
-        if (F.isEmpty(entries))
-            return Collections.emptySet();
-        else {
-            Collection<IgfsFile> res = U.newHashSet(entries.length);
-
-            for (File entry : entries) {
-                IgfsFile info = info(igfsPath(entry));
-
-                if (info != null)
-                    res.add(info);
-            }
-
-            return res;
-        }
-    }
-
-    /**
-     * Returns an array of File object. Under the specific path.
-     *
-     * @param path IGFS path.
-     * @return Array of File objects.
-     */
-    @Nullable private File[] listFiles0(IgfsPath path) {
-        File f = fileForPath(path);
-
-        if (!f.exists())
-            throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
-        else
-            return f.listFiles();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) {
-        try {
-            FileInputStream in = new FileInputStream(fileForPath(path));
-
-            return new LocalIgfsSecondaryFileSystemPositionedReadable(in, bufSize);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to open file for read: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream create(IgfsPath path, boolean overwrite) {
-        return create0(path, overwrite, DFLT_BUF_SIZE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
-        long blockSize, @Nullable Map<String, String> props) {
-        return create0(path, overwrite, bufSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) {
-        try {
-            File file = fileForPath(path);
-
-            boolean exists = file.exists();
-
-            if (exists)
-                return new BufferedOutputStream(new FileOutputStream(file, true), bufSize);
-            else {
-                if (create)
-                    return create0(path, false, bufSize);
-                else
-                    throw new IgfsPathNotFoundException("Failed to append to file because it doesn't exist: " + path);
-            }
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to append to file because it doesn't exist: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(final IgfsPath path) {
-        File f = fileForPath(path);
-
-        if (!f.exists())
-            return null;
-
-        boolean isDir = f.isDirectory();
-
-        if (isDir)
-            return new LocalFileSystemIgfsFile(path, false, true, 0, f.lastModified(), 0, null);
-        else
-            return new LocalFileSystemIgfsFile(path, f.isFile(), false, 0, f.lastModified(), f.length(), null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long usedSpaceSize() {
-        throw new UnsupportedOperationException("usedSpaceSize operation is not yet supported.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteException {
-        if (workDir != null)
-            workDir = new File(workDir).getAbsolutePath();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop() throws IgniteException {
-        // No-op.
-    }
-
-    /**
-     * Get work directory.
-     *
-     * @return Work directory.
-     */
-    @Nullable public String getWorkDirectory() {
-        return workDir;
-    }
-
-    /**
-     * Set work directory.
-     *
-     * @param workDir Work directory.
-     */
-    public void setWorkDirectory(@Nullable String workDir) {
-        this.workDir = workDir;
-    }
-
-    /**
-     * Create file for IGFS path.
-     *
-     * @param path IGFS path.
-     * @return File object.
-     */
-    private File fileForPath(IgfsPath path) {
-        if (workDir == null)
-            return new File(path.toString());
-        else {
-            if ("/".equals(path.toString()))
-                return new File(workDir);
-            else
-                return new File(workDir, path.toString());
-        }
-    }
-
-    /**
-     * Create IGFS path for file.
-     *
-     * @param f File object.
-     * @return IFGS path.
-     * @throws IgfsException If failed.
-     */
-    private IgfsPath igfsPath(File f) throws IgfsException {
-        String path = f.getAbsolutePath();
-
-        if (workDir != null) {
-            if (!path.startsWith(workDir))
-                throw new IgfsException("Path is not located in the work directory [workDir=" + workDir +
-                    ", path=" + path + ']');
-
-            path = path.substring(workDir.length(), path.length());
-        }
-
-        return new IgfsPath(path);
-    }
-
-    /**
-     * Internal create routine.
-     *
-     * @param path Path.
-     * @param overwrite Overwirte flag.
-     * @param bufSize Buffer size.
-     * @return Output stream.
-     */
-    private OutputStream create0(IgfsPath path, boolean overwrite, int bufSize) {
-        File file = fileForPath(path);
-
-        boolean exists = file.exists();
-
-        if (exists) {
-            if (!overwrite)
-                throw new IgfsPathAlreadyExistsException("Failed to create a file because it already exists: " + path);
-        }
-        else {
-            File parent = file.getParentFile();
-
-            if (!mkdirs0(parent))
-                throw new IgfsException("Failed to create parent directory for file (underlying file system " +
-                    "returned false): " + path);
-        }
-
-        try {
-            return new BufferedOutputStream(new FileOutputStream(file), bufSize);
-        }
-        catch (IOException e) {
-            throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + ']');
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystemPositionedReadable.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystemPositionedReadable.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystemPositionedReadable.java
deleted file mode 100644
index a37baaa..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/LocalIgfsSecondaryFileSystemPositionedReadable.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.hadoop.fs;
-
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadable;
-
-import java.io.BufferedInputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-
-/**
- * Positioned readable interface for local secondary file system.
- */
-public class LocalIgfsSecondaryFileSystemPositionedReadable extends BufferedInputStream
-    implements IgfsSecondaryFileSystemPositionedReadable {
-    /** Last read position. */
-    private long lastReadPos;
-
-    /**
-     * Constructor.
-     *
-     * @param in Input stream.
-     * @param bufSize Buffer size.
-     */
-    public LocalIgfsSecondaryFileSystemPositionedReadable(FileInputStream in, int bufSize) {
-        super(in, bufSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read(long readPos, byte[] buf, int off, int len) throws IOException {
-        if (in == null)
-            throw new IOException("Stream is closed.");
-
-        if (readPos < lastReadPos || readPos + len > lastReadPos + this.buf.length) {
-            ((FileInputStream)in).getChannel().position(readPos);
-
-            pos = 0;
-            count = 0;
-        }
-
-        int bytesRead = read(buf, off, len);
-
-        if (bytesRead != -1) {
-            // Advance last read position only if we really read some bytes from the stream.
-            lastReadPos = readPos + bytesRead;
-        }
-
-        return bytesRead;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalIgfsSecondaryFileSystemTestAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalIgfsSecondaryFileSystemTestAdapter.java
deleted file mode 100644
index a43b627..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalIgfsSecondaryFileSystemTestAdapter.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-import org.apache.ignite.internal.processors.igfs.IgfsEx;
-import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter;
-import org.apache.ignite.internal.util.typedef.T2;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.DirectoryStream;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.nio.file.StandardOpenOption;
-import java.util.Map;
-
-/**
- * Adapter for local secondary file system.
- */
-public class LocalIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter {
-    /** */
-    private final String workDir;
-
-    /**
-     * @param workDir Work dir.
-     */
-    public LocalIgfsSecondaryFileSystemTestAdapter(final File workDir) {
-        this.workDir = workDir.getAbsolutePath();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() throws IOException {
-        return "local";
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean exists(final String path) throws IOException {
-        return Files.exists(path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(final String path, final boolean recursive) throws IOException {
-        if (recursive)
-            return deleteRecursively(path(path));
-        else
-            return path(path).toFile().delete();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(final String path) throws IOException {
-        Files.createDirectory(path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void format() throws IOException {
-        try (DirectoryStream<Path> stream = Files.newDirectoryStream(Paths.get(workDir))) {
-            for (Path innerPath : stream)
-                deleteRecursively(innerPath);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, String> properties(final String path) throws IOException {
-        // TODO: Create ticket for this!
-        throw new UnsupportedOperationException("properties");
-    }
-
-    /** {@inheritDoc} */
-    @Override public String permissions(String path) throws IOException {
-        // TODO: Create ticket for this!
-        throw new UnsupportedOperationException("permissions");
-    }
-
-    /** {@inheritDoc} */
-    @Override public InputStream openInputStream(final String path) throws IOException {
-        return Files.newInputStream(path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream openOutputStream(final String path, final boolean append) throws IOException {
-        if (append)
-            return Files.newOutputStream(path(path), StandardOpenOption.APPEND);
-        else
-            return Files.newOutputStream(path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public T2<Long, Long> times(String path) throws IOException {
-        // TODO: Create ticket for this!
-        throw new UnsupportedOperationException("times");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsEx igfs() {
-        return null;
-    }
-
-    /**
-     * Convert path.
-     *
-     * @param path String path.
-     * @return Java File API path.
-     */
-    private Path path(String path) {
-        return Paths.get(workDir + path);
-    }
-
-    /**
-     * Delete recursively.
-     *
-     * @param path Path.
-     * @throws IOException If failed.
-     */
-    private boolean deleteRecursively(Path path) throws IOException {
-        if (Files.isDirectory(path)) {
-            try (DirectoryStream<Path> stream = Files.newDirectoryStream(path)) {
-                for (Path innerPath : stream) {
-                    boolean res = deleteRecursively(innerPath);
-
-                    if (!res)
-                        return false;
-                }
-            }
-        }
-
-        return path.toFile().delete();
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAbstractSelfTest.java
deleted file mode 100644
index 8efe2c9..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAbstractSelfTest.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-import org.apache.ignite.hadoop.fs.LocalIgfsSecondaryFileSystem;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
-import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest;
-import org.apache.ignite.internal.util.typedef.internal.U;
-
-import java.io.File;
-
-/**
- * Abstract test for Hadoop 1.0 file system stack.
- */
-public abstract class LocalSecondaryFileSystemDualAbstractSelfTest extends IgfsDualAbstractSelfTest {
-    /** */
-    private static final String FS_WORK_DIR = U.getIgniteHome() + File.separatorChar + "work"
-        + File.separatorChar + "fs";
-
-    /** Constructor.
-     * @param mode IGFS mode.
-     */
-    public LocalSecondaryFileSystemDualAbstractSelfTest(IgfsMode mode) {
-        super(mode);
-    }
-
-    /**
-     * Creates secondary filesystems.
-     * @return IgfsSecondaryFileSystem
-     * @throws Exception On failure.
-     */
-    @Override protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
-       final File workDir = new File(FS_WORK_DIR);
-
-        if (!workDir.exists())
-            assert workDir.mkdirs();
-
-        LocalIgfsSecondaryFileSystem second = new LocalIgfsSecondaryFileSystem();
-
-        second.setWorkDirectory(workDir.getAbsolutePath());
-
-        igfsSecondary = new LocalIgfsSecondaryFileSystemTestAdapter(workDir);
-
-        return second;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean permissionsSupported() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean propertiesSupported() {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean timesSupported() {
-        return false;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncClientSelfTest.java
deleted file mode 100644
index 56ef45e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncClientSelfTest.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-/**
- * DUAL_ASYNC mode.
- */
-public class LocalSecondaryFileSystemDualAsyncClientSelfTest extends LocalSecondaryFileSystemDualAsyncSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean client() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncSelfTest.java
deleted file mode 100644
index b8c82b2..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualAsyncSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-/**
- * DUAL_ASYNC mode test.
- */
-public class LocalSecondaryFileSystemDualAsyncSelfTest extends LocalSecondaryFileSystemDualAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public LocalSecondaryFileSystemDualAsyncSelfTest() {
-        super(IgfsMode.DUAL_ASYNC);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncClientSelfTest.java
deleted file mode 100644
index bc73f4d..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncClientSelfTest.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-/**
- * DUAL_SYNC mode.
- */
-public class LocalSecondaryFileSystemDualSyncClientSelfTest extends LocalSecondaryFileSystemDualSyncSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean client() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/e0771b88/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncSelfTest.java
deleted file mode 100644
index 6ff98c2..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/LocalSecondaryFileSystemDualSyncSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-/**
- * DUAL_SYNC mode.
- */
-public class LocalSecondaryFileSystemDualSyncSelfTest extends LocalSecondaryFileSystemDualAbstractSelfTest {
-    /**
-     * Constructor.
-     */
-    public LocalSecondaryFileSystemDualSyncSelfTest() {
-        super(IgfsMode.DUAL_SYNC);
-    }
-}
\ No newline at end of file