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 14:18:27 UTC

[11/15] ignite git commit: IGNITE-1926: IGFS: Implemented local secondary file system.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/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/5cf3bea3/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java
new file mode 100644
index 0000000..fe913df
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemTestAdapter.java
@@ -0,0 +1,118 @@
+/*
+ * 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 org.jetbrains.annotations.Nullable;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Map;
+
+/**
+ * Universal interface to an underlying filesystem.
+ * Typically used for secondary filesystem.
+ * To be used solely in tests.
+ */
+public interface IgfsSecondaryFileSystemTestAdapter {
+    /**
+     * Gets name of the FS.
+     * @return name of this file system.
+     * @throws IOException in case of failure.
+     */
+    String name() throws IOException;
+
+    /**
+     * Answers if a file denoted by path exists.
+     * @param path path of the file to check.
+     * @return if the file exists.
+     * @throws IOException in case of failure.
+     */
+    boolean exists(String path) throws IOException;
+
+    /**
+     * Deletes a file or directory.
+     * @param path the path to delete.
+     * @param recursive instructs to delete a directory recursively.
+     * @return true on success, false otherwise.
+     * @throws IOException On failure.
+     */
+    boolean delete(String path, boolean recursive) throws IOException;
+
+    /**
+     * Makes directories, creating missing parent directories as needed.
+     * @param path the directory to create.
+     * @throws IOException On failure.
+     */
+    void mkdirs(String path) throws IOException;
+
+    /**
+     * Clears (formats) entire the filesystem.
+     * All the data in the filesystem are DESTROYED.
+     * @throws IOException On failure.
+     */
+    void format() throws IOException;
+
+    /**
+     * Gets properties (such as owner, group, and permissions) of a file.
+     * @param path the path to the file to get properties of.
+     * @return the properties.
+     */
+    Map<String,String> properties(String path) throws IOException;
+
+    /**
+     * Get permissions.
+     *
+     * @param path Path.
+     * @return Permissions.
+     * @throws IOException If failed.
+     */
+    String permissions(String path) throws IOException;
+
+    /**
+     * Opens input stream to read file contents.
+     * @param path the path to the file.
+     */
+    InputStream openInputStream(String path) throws IOException;
+
+    /**
+     * Opens output stream to write file contents.
+     * @param path the path to the file to be written.
+     * @param append if to append to the end of existing data.
+     * @return the OutputStream to write into.
+     * @throws IOException On failure.
+     */
+    OutputStream openOutputStream(String path, boolean append) throws IOException;
+
+    /**
+     * Get times for path.
+     *
+     * @param path Path.
+     * @return Times for path.
+     * @throws IOException If failed.
+     */
+    T2<Long, Long> times(String path) throws IOException;
+
+    /**
+     * Get underlying IGFS if it is possible.
+     *
+     * @return Underlying IGFS or null.
+     */
+    @Nullable IgfsEx igfs();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
deleted file mode 100644
index 3f6d07e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
+++ /dev/null
@@ -1,109 +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.internal.processors.igfs;
-
-import org.apache.ignite.internal.util.typedef.T2;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.Map;
-
-/**
- * Universal interface to an underlying filesystem.
- * Typically used for secondary filesystem.
- * To be used solely in tests.
- */
-public interface UniversalFileSystemAdapter {
-    /**
-     * Gets name of the FS.
-     * @return name of this file system.
-     * @throws IOException in case of failure.
-     */
-    String name() throws IOException;
-
-    /**
-     * Answers if a file denoted by path exists.
-     * @param path path of the file to check.
-     * @return if the file exists.
-     * @throws IOException in case of failure.
-     */
-    boolean exists(String path) throws IOException;
-
-    /**
-     * Deletes a file or directory.
-     * @param path the path to delete.
-     * @param recursive instructs to delete a directory recursively.
-     * @return true on success, false otherwise.
-     * @throws IOException On failure.
-     */
-    boolean delete(String path, boolean recursive) throws IOException;
-
-    /**
-     * Makes directories, creating missing parent directories as needed.
-     * @param path the directory to create.
-     * @throws IOException On failure.
-     */
-    void mkdirs(String path) throws IOException;
-
-    /**
-     * Clears (formats) entire the filesystem.
-     * All the data in the filesystem are DESTROYED.
-     * @throws IOException
-     */
-    void format() throws IOException;
-
-    /**
-     * Gets properties (such as owner, group, and permissions) of a file.
-     * @param path the path to the file to get properties of.
-     * @return the properties.
-     */
-    Map<String,String> properties(String path) throws IOException;
-
-    /**
-     * Opens input stream to read file contents.
-     * @param path the path to the file.
-     */
-    InputStream openInputStream(String path) throws IOException;
-
-    /**
-     * Opens output stream to write file contents.
-     * @param path the path to the file to be written.
-     * @param append if to append to the end of existing data.
-     * @return the OutputStream to write into.
-     * @throws IOException On failure.
-     */
-    OutputStream openOutputStream(String path, boolean append) throws IOException;
-
-    /**
-     * Get times for path.
-     *
-     * @param path Path.
-     * @return Times for path.
-     * @throws IOException If failed.
-     */
-    T2<Long, Long> times(String path) throws IOException;
-
-    /**
-     * Gets an entity of the given type (class) associated with this universal adapter.
-     * @param clazz The class representing the type we wish to adapt to.
-     * @param <T> The type we need to adapt to.
-     * @return the adapter object of the given type.
-     */
-    <T> T unwrap(Class<T> clazz);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/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/5cf3bea3/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index 0f17fa2..6b5c776 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -570,7 +570,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public void stop() throws IgniteException {
         if (fsFactory instanceof LifecycleAware)
-             ((LifecycleAware)fsFactory).stop();
+            ((LifecycleAware)fsFactory).stop();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index 6b83a3e..2c25a06 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -27,7 +27,6 @@ import org.apache.ignite.hadoop.util.KerberosUserNameMapper;
 import org.apache.ignite.hadoop.util.UserNameMapper;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.jetbrains.annotations.Nullable;
 
@@ -101,7 +100,7 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
 
         second.setFileSystemFactory(factory);
 
-        igfsSecondary = new HadoopFileSystemUniversalFileSystemAdapter(factory);
+        igfsSecondary = new HadoopIgfsSecondaryFileSystemTestAdapter(factory);
 
         return second;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
deleted file mode 100644
index 5239054..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
+++ /dev/null
@@ -1,139 +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 java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.ignite.configuration.FileSystemConfiguration;
-import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory;
-import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils;
-import org.apache.ignite.internal.processors.igfs.IgfsUtils;
-import org.apache.ignite.internal.processors.igfs.UniversalFileSystemAdapter;
-import org.apache.ignite.internal.util.typedef.T2;
-
-/**
- * Universal adapter wrapping {@link org.apache.hadoop.fs.FileSystem} instance.
- */
-public class HadoopFileSystemUniversalFileSystemAdapter implements UniversalFileSystemAdapter {
-    /** File system factory. */
-    private final HadoopFileSystemFactory factory;
-
-    /**
-     * Constructor.
-     * @param factory File system factory.
-     */
-    public HadoopFileSystemUniversalFileSystemAdapter(HadoopFileSystemFactory factory) {
-        assert factory != null;
-
-        this.factory = factory;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() throws IOException {
-        return get().getUri().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean exists(String path) throws IOException {
-        return get().exists(new Path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(String path, boolean recursive) throws IOException {
-        return get().delete(new Path(path), recursive);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void mkdirs(String path) throws IOException {
-        boolean ok = get().mkdirs(new Path(path));
-        if (!ok)
-            throw new IOException("Failed to mkdirs: " + path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void format() throws IOException {
-        HadoopIgfsUtils.clear(get());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, String> properties(String path) throws IOException {
-        Path p = new Path(path);
-
-        FileStatus status = get().getFileStatus(p);
-
-        Map<String,String> m = new HashMap<>(3); // max size == 4
-
-        m.put(IgfsUtils.PROP_USER_NAME, status.getOwner());
-        m.put(IgfsUtils.PROP_GROUP_NAME, status.getGroup());
-
-        FsPermission perm = status.getPermission();
-
-        m.put(IgfsUtils.PROP_PERMISSION, "0" + perm.getUserAction().ordinal() + perm.getGroupAction().ordinal() +
-            perm.getOtherAction().ordinal());
-
-        return m;
-    }
-
-    /** {@inheritDoc} */
-    @Override public InputStream openInputStream(String path) throws IOException {
-        return get().open(new Path(path));
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputStream openOutputStream(String path, boolean append) throws IOException {
-        Path p = new Path(path);
-
-        if (append)
-            return get().append(p);
-        else
-            return get().create(p, true/*overwrite*/);
-    }
-
-    /** {@inheritDoc} */
-    @Override public T2<Long, Long> times(String path) throws IOException {
-        FileStatus status = get().getFileStatus(new Path(path));
-
-        return new T2<>(status.getAccessTime(), status.getModificationTime());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> T unwrap(Class<T> cls) {
-        if (HadoopFileSystemFactory.class.isAssignableFrom(cls))
-            return (T)factory;
-
-        return null;
-    }
-
-    /**
-     * Create file system.
-     *
-     * @return File system.
-     * @throws IOException If failed.
-     */
-    private FileSystem get() throws IOException {
-        return factory.get(FileSystemConfiguration.DFLT_USER_NAME);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5cf3bea3/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
new file mode 100644
index 0000000..f7af6f0
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsSecondaryFileSystemTestAdapter.java
@@ -0,0 +1,149 @@
+/*
+ * 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 java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.ignite.configuration.FileSystemConfiguration;
+import org.apache.ignite.hadoop.fs.HadoopFileSystemFactory;
+import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils;
+import org.apache.ignite.internal.processors.igfs.IgfsEx;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.processors.igfs.IgfsSecondaryFileSystemTestAdapter;
+import org.apache.ignite.internal.util.typedef.T2;
+
+/**
+ * Universal adapter wrapping {@link org.apache.hadoop.fs.FileSystem} instance.
+ */
+public class HadoopIgfsSecondaryFileSystemTestAdapter implements IgfsSecondaryFileSystemTestAdapter {
+    /** File system factory. */
+    private final HadoopFileSystemFactory factory;
+
+    /**
+     * Constructor.
+     * @param factory File system factory.
+     */
+    public HadoopIgfsSecondaryFileSystemTestAdapter(HadoopFileSystemFactory factory) {
+        assert factory != null;
+
+        this.factory = factory;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() throws IOException {
+        return get().getUri().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean exists(String path) throws IOException {
+        return get().exists(new Path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(String path, boolean recursive) throws IOException {
+        return get().delete(new Path(path), recursive);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(String path) throws IOException {
+        boolean ok = get().mkdirs(new Path(path));
+        if (!ok)
+            throw new IOException("Failed to mkdirs: " + path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void format() throws IOException {
+        HadoopIgfsUtils.clear(get());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, String> properties(String path) throws IOException {
+        Path p = new Path(path);
+
+        FileStatus status = get().getFileStatus(p);
+
+        Map<String,String> m = new HashMap<>(3);
+
+        m.put(IgfsUtils.PROP_USER_NAME, status.getOwner());
+        m.put(IgfsUtils.PROP_GROUP_NAME, status.getGroup());
+        m.put(IgfsUtils.PROP_PERMISSION, permission(status));
+
+        return m;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String permissions(String path) throws IOException {
+        return permission(get().getFileStatus(new Path(path)));
+    }
+
+    /**
+     * Get permission for file status.
+     *
+     * @param status Status.
+     * @return Permission.
+     */
+    private String permission(FileStatus status) {
+        FsPermission perm = status.getPermission();
+
+        return "0" + perm.getUserAction().ordinal() + perm.getGroupAction().ordinal() + perm.getOtherAction().ordinal();
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputStream openInputStream(String path) throws IOException {
+        return get().open(new Path(path));
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream openOutputStream(String path, boolean append) throws IOException {
+        Path p = new Path(path);
+
+        if (append)
+            return get().append(p);
+        else
+            return get().create(p, true/*overwrite*/);
+    }
+
+    /** {@inheritDoc} */
+    @Override public T2<Long, Long> times(String path) throws IOException {
+        FileStatus status = get().getFileStatus(new Path(path));
+
+        return new T2<>(status.getAccessTime(), status.getModificationTime());
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsEx igfs() {
+        return null;
+    }
+
+    /**
+     * Create file system.
+     *
+     * @return File system.
+     * @throws IOException If failed.
+     */
+    protected FileSystem get() throws IOException {
+        return factory.get(FileSystemConfiguration.DFLT_USER_NAME);
+    }
+}
\ No newline at end of file