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 2015/12/23 16:25:37 UTC

[2/4] ignite git commit: IGNITE-2206: Decoupled caching and non-caching impls.

IGNITE-2206: Decoupled caching and non-caching impls.


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

Branch: refs/heads/ignite-2206
Commit: 262af69886387dece31dfe76d3cedfa48a20e384
Parents: 0effa10
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Dec 23 18:11:43 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Dec 23 18:11:43 2015 +0300

----------------------------------------------------------------------
 .../hadoop/fs/BasicHadoopFileSystemFactory.java | 189 +++++++++++++++++++
 .../fs/CachingHadoopFileSystemFactory.java      | 171 ++---------------
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   2 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   |   8 +-
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +-
 ...oopSecondaryFileSystemConfigurationTest.java |  16 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  52 ++---
 7 files changed, 247 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
new file mode 100644
index 0000000..a2317cf
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/BasicHadoopFileSystemFactory.java
@@ -0,0 +1,189 @@
+/*
+ * 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.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.processors.hadoop.HadoopUtils;
+import org.apache.ignite.internal.processors.igfs.IgfsUtils;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lifecycle.LifecycleAware;
+import org.jetbrains.annotations.Nullable;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.util.Arrays;
+
+/**
+ * Caching Hadoop file system factory. File systems are cache on per-user basis.
+ */
+public class BasicHadoopFileSystemFactory implements HadoopFileSystemFactory, Externalizable, LifecycleAware {
+    /** File system URI. */
+    protected String uri;
+
+    /** File system config paths. */
+    protected String[] cfgPaths;
+
+    /** Configuration of the secondary filesystem, never null. */
+    protected Configuration cfg;
+
+    /** */
+    protected URI fullUri;
+
+    /**
+     * Public non-arg constructor.
+     */
+    public BasicHadoopFileSystemFactory() {
+        // noop
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileSystem create(String usrName) throws IOException {
+        return create0(IgfsUtils.fixUserName(usrName));
+    }
+
+    /**
+     * Internal file system create routine.
+     *
+     * @param usrName User name.
+     * @return File system.
+     * @throws IOException If failed.
+     */
+    protected FileSystem create0(String usrName) throws IOException {
+        assert cfg != null;
+
+        try {
+            return FileSystem.get(fullUri, cfg, usrName);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IOException("Failed to create file system due to interrupt.", e);
+        }
+    }
+
+    /**
+     * Gets file system URI.
+     *
+     * @return File system URI.
+     */
+    @Nullable public String getUri() {
+        return uri;
+    }
+
+    /**
+     * Sets file system URI.
+     *
+     * @param uri File system URI.
+     */
+    public void setUri(@Nullable String uri) {
+        this.uri = uri;
+    }
+
+    /**
+     * Gets paths to additional file system configuration files (e.g. core-site.xml).
+     *
+     * @return Paths to file system configuration files.
+     */
+    @Nullable public String[] getConfigPaths() {
+        return cfgPaths;
+    }
+
+    /**
+     * Set paths to additional file system configuration files (e.g. core-site.xml).
+     *
+     * @param cfgPaths Paths to file system configuration files.
+     */
+    public void setConfigPaths(String... cfgPaths) {
+        this.cfgPaths = cfgPaths;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        // if secondary fs URI is not given explicitly, try to get it from the configuration:
+        if (uri == null)
+            fullUri = FileSystem.getDefaultUri(cfg);
+        else {
+            try {
+                fullUri = new URI(uri);
+            }
+            catch (URISyntaxException use) {
+                throw new IgniteException("Failed to resolve secondary file system URI: " + uri);
+            }
+        }
+
+        cfg = HadoopUtils.safeCreateConfiguration();
+
+        if (cfgPaths != null) {
+            for (String cfgPath : cfgPaths) {
+                if (cfgPath == null)
+                    throw new IgniteException("Configuration path cannot be null: " + Arrays.toString(cfgPaths));
+                else {
+                    URL url = U.resolveIgniteUrl(cfgPath);
+
+                    if (url == null) {
+                        // If secConfPath is given, it should be resolvable:
+                        throw new IgniteException("Failed to resolve secondary file system configuration path " +
+                            "(ensure that it exists locally and you have read access to it): " + cfgPath);
+                    }
+
+                    cfg.addResource(url);
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, uri);
+
+        if (cfgPaths != null) {
+            out.writeInt(cfgPaths.length);
+
+            for (String cfgPath : cfgPaths)
+                U.writeString(out, cfgPath);
+        }
+        else
+            out.writeInt(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        uri = U.readString(in);
+
+        int cfgPathsCnt = in.readInt();
+
+        if (cfgPathsCnt != -1) {
+            cfgPaths = new String[cfgPathsCnt];
+
+            for (int i = 0; i < cfgPathsCnt; i++)
+                cfgPaths[i] = U.readString(in);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
index 52d4db5..baad1fc 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/CachingHadoopFileSystemFactory.java
@@ -17,62 +17,28 @@
 
 package org.apache.ignite.hadoop.fs;
 
-import java.io.Externalizable;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.hadoop.HadoopUtils;
 import org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemsUtils;
 import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap;
-import org.apache.ignite.internal.processors.igfs.IgfsPaths;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.A;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.lifecycle.LifecycleAware;
+
+import java.io.IOException;
 
 /**
- * The class is to be instantiated as a Spring beans, so it must have public zero-arg constructor.
- * The class is serializable as it will be transferred over the network as a part of {@link IgfsPaths} object.
+ * Caching Hadoop file system factory. File systems are cache on per-user basis.
  */
-public class CachingHadoopFileSystemFactory implements HadoopFileSystemFactory, Externalizable, LifecycleAware {
+public class CachingHadoopFileSystemFactory extends BasicHadoopFileSystemFactory {
     /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
-    private final transient HadoopLazyConcurrentMap<String, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
+    private final transient HadoopLazyConcurrentMap<String, FileSystem> cache = new HadoopLazyConcurrentMap<>(
         new HadoopLazyConcurrentMap.ValueFactory<String, FileSystem>() {
-            @Override public FileSystem createValue(String key) {
-                try {
-                    assert !F.isEmpty(key);
-
-                    return createFileSystem(key);
-                }
-                catch (IOException ioe) {
-                    throw new IgniteException(ioe);
-                }
+            @Override public FileSystem createValue(String key) throws IOException {
+                return create0(key);
             }
         }
     );
 
-    /** Configuration of the secondary filesystem, never null. */
-    protected transient Configuration cfg;
-
-    /** */
-    protected transient URI uri;
-
-    /** */
-    protected String uriStr;
-
-    /** */
-    protected List<String> cfgPathStr;
-
     /**
      * Public non-arg constructor.
      */
@@ -81,131 +47,24 @@ public class CachingHadoopFileSystemFactory implements HadoopFileSystemFactory,
     }
 
     /** {@inheritDoc} */
-    @Override public FileSystem create(String userName) throws IOException {
-        A.ensure(cfg != null, "cfg");
-
-        return fileSysLazyMap.getOrCreate(userName);
-    }
-
-    /**
-     * Uri setter.
-     *
-     * @param uriStr The URI to set.
-     */
-    public void setUri(String uriStr) {
-        this.uriStr = uriStr;
-    }
-
-    /**
-     * Gets the URI.
-     *
-     * @return The URI.
-     */
-    public URI getUri() {
-        return uri;
-    }
-
-    /**
-     * Configuration(s) setter, to be invoked from Spring config.
-     *
-     * @param cfgPaths The config paths collection to set.
-     */
-    public void setConfigPaths(List<String> cfgPaths) {
-        this.cfgPathStr = cfgPaths;
-    }
-
-    /**
-     * Gets the config paths collection.
-     *
-     * @return The config paths collection.
-     */
-    public List<String> getConfigPaths() {
-        return cfgPathStr;
-    }
-
-    /**
-     * @return {@link org.apache.hadoop.fs.FileSystem}  instance for this secondary Fs.
-     * @throws IOException
-     */
-    protected FileSystem createFileSystem(String userName) throws IOException {
-        userName = IgfsUtils.fixUserName(userName);
-
-        assert cfg != null;
-
-        final FileSystem fileSys;
-
-        try {
-            fileSys = FileSystem.get(uri, cfg, userName);
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IOException("Failed to create file system due to interrupt.", e);
-        }
-
-        return fileSys;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeString(out, uriStr);
-
-        U.writeCollection(out, cfgPathStr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        uriStr = U.readString(in);
-
-        cfgPathStr = new ArrayList(U.readCollection(in));
+    @Override public FileSystem create(String usrName) throws IOException {
+        return cache.getOrCreate(IgfsUtils.fixUserName(usrName));
     }
 
     /** {@inheritDoc} */
     @Override public void start() throws IgniteException {
-        cfg = HadoopUtils.safeCreateConfiguration();
-
-        if (cfgPathStr != null) {
-            for (String confPath : cfgPathStr) {
-                if (confPath == null)
-                    throw new IgniteException("Null config path encountered.");
-                else {
-                    URL url = U.resolveIgniteUrl(confPath);
-
-                    if (url == null) {
-                        // If secConfPath is given, it should be resolvable:
-                        throw new IgniteException("Failed to resolve secondary file system configuration path " +
-                            "(ensure that it exists locally and you have read access to it): " + confPath);
-                    }
-
-                    cfg.addResource(url);
-                }
-            }
-        }
+        super.start();
 
-        // if secondary fs URI is not given explicitly, try to get it from the configuration:
-        if (uriStr == null)
-            uri = FileSystem.getDefaultUri(cfg);
-        else {
-            try {
-                uri = new URI(uriStr);
-            }
-            catch (URISyntaxException use) {
-                throw new IgniteException("Failed to resolve secondary file system URI: " + uriStr);
-            }
-        }
-
-        assert uriStr != null;
-
-        // Disable caching:
-        String prop = HadoopFileSystemsUtils.disableFsCachePropertyName(uri.getScheme());
-
-        cfg.setBoolean(prop, true);
+        // Disable caching.
+        cfg.setBoolean(HadoopFileSystemsUtils.disableFsCachePropertyName(fullUri.getScheme()), true);
     }
 
     /** {@inheritDoc} */
     @Override public void stop() throws IgniteException {
+        super.stop();
+
         try {
-            fileSysLazyMap.close();
+            cache.close();
         }
         catch (IgniteCheckedException ice) {
             throw new IgniteException(ice);

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/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 9942ec4..99b8f5a 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
@@ -122,7 +122,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         fac.setUri(uri);
 
         if (cfgPath != null)
-            fac.setConfigPaths(Collections.singletonList(cfgPath));
+            fac.setConfigPaths(cfgPath);
 
         setFileSystemFactory(fac);
         setDefaultUserName(userName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
index 48ade79..1ecbee5 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
@@ -39,7 +39,7 @@ public class HadoopFileSystemCacheUtils {
     public static HadoopLazyConcurrentMap<FsCacheKey, FileSystem> createHadoopLazyConcurrentMap() {
         return new HadoopLazyConcurrentMap<>(
             new HadoopLazyConcurrentMap.ValueFactory<FsCacheKey, FileSystem>() {
-                @Override public FileSystem createValue(FsCacheKey key) {
+                @Override public FileSystem createValue(FsCacheKey key) throws IOException {
                     try {
                         assert key != null;
 
@@ -57,8 +57,10 @@ public class HadoopFileSystemCacheUtils {
 
                         return FileSystem.get(uri, cfg, key.user());
                     }
-                    catch (IOException | InterruptedException ioe) {
-                        throw new IgniteException(ioe);
+                    catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+
+                        throw new IOException("Failed to create file system due to interrupt.", e);
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
index 5808d01..e399a4a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.fs;
 
 import java.io.Closeable;
+import java.io.IOException;
 import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -204,8 +205,8 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
          *
          * @param key the key to create value for
          * @return the value.
-         * @throws IgniteException on failure.
+         * @throws IOException On failure.
          */
-        public V createValue(K key);
+        public V createValue(K key) throws IOException;
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index 98ab317..59bf7ee 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -17,13 +17,6 @@
 
 package org.apache.ignite.igfs;
 
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collections;
-import java.util.concurrent.Callable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
@@ -49,6 +42,13 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.util.concurrent.Callable;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -176,7 +176,7 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
 
         CachingHadoopFileSystemFactory fac = new CachingHadoopFileSystemFactory();
 
-        fac.setConfigPaths(Collections.singletonList(primaryConfFullPath));
+        fac.setConfigPaths(primaryConfFullPath);
         fac.setUri(primaryFsUriStr);
 
         fac.start();

http://git-wip-us.apache.org/repos/asf/ignite/blob/262af698/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index d473592..14ff25d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -17,30 +17,6 @@
 
 package org.apache.ignite.igfs;
 
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.lang.reflect.Field;
-import java.net.URI;
-import java.security.PrivilegedExceptionAction;
-import java.util.ArrayDeque;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Deque;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.ContentSummary;
@@ -60,9 +36,9 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
 import org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem;
-import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEx;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsIpcIo;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsOutProc;
@@ -81,6 +57,30 @@ import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ThreadLocalRandom8;
 
+import java.io.BufferedOutputStream;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.lang.reflect.Field;
+import java.net.URI;
+import java.security.PrivilegedExceptionAction;
+import java.util.ArrayDeque;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Deque;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -386,7 +386,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
             CachingHadoopFileSystemFactory fac = new CachingHadoopFileSystemFactory();
 
             fac.setUri(SECONDARY_URI);
-            fac.setConfigPaths(Collections.singletonList(SECONDARY_CFG_PATH));
+            fac.setConfigPaths(SECONDARY_CFG_PATH);
 
             IgniteHadoopIgfsSecondaryFileSystem sec = new IgniteHadoopIgfsSecondaryFileSystem();