You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/03/05 10:06:52 UTC

[41/51] incubator-ignite git commit: IGNITE-386: Squashed changes.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInput.java
new file mode 100644
index 0000000..ad6446f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInput.java
@@ -0,0 +1,55 @@
+/*
+ * 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.hadoop;
+
+import org.apache.ignite.*;
+
+import java.util.*;
+
+/**
+ * Task input.
+ */
+public interface HadoopTaskInput extends AutoCloseable {
+    /**
+     * Moves cursor to the next element.
+     *
+     * @return {@code false} If input is exceeded.
+     */
+    boolean next();
+
+    /**
+     * Gets current key.
+     *
+     * @return Key.
+     */
+    Object key();
+
+    /**
+     * Gets values for current key.
+     *
+     * @return Values.
+     */
+    Iterator<?> values();
+
+    /**
+     * Closes input.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    @Override public void close() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskOutput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskOutput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskOutput.java
new file mode 100644
index 0000000..41d9847
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskOutput.java
@@ -0,0 +1,40 @@
+/*
+ * 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.hadoop;
+
+import org.apache.ignite.*;
+
+/**
+ * Task output.
+ */
+public interface HadoopTaskOutput extends AutoCloseable {
+    /**
+     * Writes key and value to the output.
+     *
+     * @param key Key.
+     * @param val Value.
+     */
+    public void write(Object key, Object val) throws IgniteCheckedException;
+
+    /**
+     * Closes output.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    @Override public void close() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskType.java
new file mode 100644
index 0000000..a88e189
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskType.java
@@ -0,0 +1,56 @@
+/*
+ * 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.hadoop;
+
+import org.jetbrains.annotations.*;
+
+/**
+* Task type.
+*/
+public enum HadoopTaskType {
+    /** Setup task. */
+    SETUP,
+
+    /** Map task. */
+    MAP,
+
+    /** Reduce task. */
+    REDUCE,
+
+    /** Combine task. */
+    COMBINE,
+
+    /** Commit task. */
+    COMMIT,
+
+    /** Abort task. */
+    ABORT;
+
+    /** Enumerated values. */
+    private static final HadoopTaskType[] VALS = values();
+
+    /**
+     * Efficiently gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value.
+     */
+    @Nullable public static HadoopTaskType fromOrdinal(byte ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
deleted file mode 100644
index d0ef4ce..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
+++ /dev/null
@@ -1,74 +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.hadoop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.future.*;
-
-/**
- * Hadoop processor.
- */
-public class IgniteHadoopNoopProcessor extends IgniteHadoopProcessorAdapter {
-    /**
-     * @param ctx Kernal context.
-     */
-    public IgniteHadoopNoopProcessor(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoop hadoop() {
-        throw new IllegalStateException("Hadoop module is not found in class path.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration config() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobId nextJobId() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
-        return new GridFinishedFutureEx<>(new IgniteCheckedException("Hadoop is not available."));
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopCounters counters(GridHadoopJobId jobId) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return false;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
deleted file mode 100644
index c2cf542..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
+++ /dev/null
@@ -1,94 +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.hadoop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.*;
-
-/**
- * Hadoop processor.
- */
-public abstract class IgniteHadoopProcessorAdapter extends GridProcessorAdapter {
-    /**
-     * @param ctx Kernal context.
-     */
-    protected IgniteHadoopProcessorAdapter(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /**
-     * @return Hadoop facade.
-     */
-    public abstract GridHadoop hadoop();
-
-    /**
-     * @return Hadoop configuration.
-     */
-    public abstract GridHadoopConfiguration config();
-
-    /**
-     * @return Collection of generated IDs.
-     */
-    public abstract GridHadoopJobId nextJobId();
-
-    /**
-     * Submits job to job tracker.
-     *
-     * @param jobId Job ID to submit.
-     * @param jobInfo Job info to submit.
-     * @return Execution future.
-     */
-    public abstract IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
-
-    /**
-     * Gets Hadoop job execution status.
-     *
-     * @param jobId Job ID to get status for.
-     * @return Job execution status.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Returns Hadoop job counters.
-     *
-     * @param jobId Job ID to get counters for.
-     * @return Job counters.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Gets Hadoop job finish future.
-     *
-     * @param jobId Job ID.
-     * @return Job finish future or {@code null}.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * Kills job.
-     *
-     * @param jobId Job ID.
-     * @return {@code True} if job was killed.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounter.java
new file mode 100644
index 0000000..918c3bc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.hadoop.counter;
+
+/**
+ * Hadoop counter.
+ */
+public interface HadoopCounter {
+    /**
+     * Gets name.
+     *
+     * @return Name of the counter.
+     */
+    public String name();
+
+    /**
+     * Gets counter group.
+     *
+     * @return Counter group's name.
+     */
+    public String group();
+
+    /**
+     * Merge the given counter to this counter.
+     *
+     * @param cntr Counter to merge into this counter.
+     */
+    public void merge(HadoopCounter cntr);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
new file mode 100644
index 0000000..ce67c57
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
@@ -0,0 +1,37 @@
+/*
+ * 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.hadoop.counter;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+/**
+ * The object that writes some system counters to some storage for each running job. This operation is a part of
+ * whole statistics collection process.
+ */
+public interface HadoopCounterWriter {
+    /**
+     * Writes counters of given job to some statistics storage.
+     *
+     * @param jobInfo Job info.
+     * @param jobId Job id.
+     * @param cntrs Counters.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounters.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounters.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounters.java
new file mode 100644
index 0000000..706ba77
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounters.java
@@ -0,0 +1,49 @@
+/*
+ * 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.hadoop.counter;
+
+import java.util.*;
+
+/**
+ * Counters store.
+ */
+public interface HadoopCounters {
+    /**
+     * Returns counter for the specified group and counter name. Creates new if it does not exist.
+     *
+     * @param grp Counter group name.
+     * @param name Counter name.
+     * @param cls Class for new instance creation if it's needed.
+     * @return The counter that was found or added or {@code null} if create is false.
+     */
+    <T extends HadoopCounter> T counter(String grp, String name, Class<T> cls);
+
+    /**
+     * Returns all existing counters.
+     *
+     * @return Collection of counters.
+     */
+    Collection<HadoopCounter> all();
+
+    /**
+     * Merges all counters from another store with existing counters.
+     *
+     * @param other Counters to merge with.
+     */
+    void merge(HadoopCounters other);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
index 1479a88..48a32f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsAsyncImpl.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.mapreduce.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -32,7 +33,7 @@ import java.util.*;
 /**
  * Igfs supporting asynchronous operations.
  */
-public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFs> implements IgfsEx {
+public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFileSystem> implements IgfsEx {
     /** */
     private final IgfsImpl igfs;
 
@@ -192,7 +193,7 @@ public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFs> implements Igfs
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsConfiguration configuration() {
+    @Override public FileSystemConfiguration configuration() {
         return igfs.configuration();
     }
 
@@ -310,7 +311,7 @@ public class IgfsAsyncImpl extends AsyncSupportAdapter<IgniteFs> implements Igfs
     }
 
     /** {@inheritDoc} */
-    @Override public Map<String, String> properties() {
-        return igfs.properties();
+    @Override public IgfsSecondaryFileSystem asSecondary() {
+        return igfs.asSecondary();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 245a1dd..475a7fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -37,7 +37,7 @@ public class IgfsContext {
     private final GridKernalContext ctx;
 
     /** Configuration. */
-    private final IgfsConfiguration cfg;
+    private final FileSystemConfiguration cfg;
 
     /** Managers. */
     private List<IgfsManager> mgrs = new LinkedList<>();
@@ -68,7 +68,7 @@ public class IgfsContext {
      */
     public IgfsContext(
         GridKernalContext ctx,
-        IgfsConfiguration cfg,
+        FileSystemConfiguration cfg,
         IgfsMetaManager metaMgr,
         IgfsDataManager dataMgr,
         IgfsServerManager srvMgr,
@@ -102,7 +102,7 @@ public class IgfsContext {
     /**
      * @return IGFS configuration.
      */
-    public IgfsConfiguration configuration() {
+    public FileSystemConfiguration configuration() {
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index e960422..72bd60a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -306,7 +307,7 @@ public class IgfsDataManager extends IgfsManager {
         IgniteDataLoader<IgfsBlockKey, byte[]> ldr =
             igfsCtx.kernalContext().<IgfsBlockKey, byte[]>dataLoad().dataLoader(dataCachePrj.name());
 
-        IgfsConfiguration cfg = igfsCtx.configuration();
+        FileSystemConfiguration cfg = igfsCtx.configuration();
 
         if (cfg.getPerNodeBatchSize() > 0)
             ldr.perNodeBufferSize(cfg.getPerNodeBatchSize());
@@ -382,7 +383,7 @@ public class IgfsDataManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     @Nullable public IgniteInternalFuture<byte[]> dataBlock(final IgfsFileInfo fileInfo, final IgfsPath path,
-        final long blockIdx, @Nullable final IgfsReader secReader)
+        final long blockIdx, @Nullable final IgfsSecondaryFileSystemPositionedReadable secReader)
         throws IgniteCheckedException {
         //assert validTxState(any); // Allow this method call for any transaction state.
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
index a380a6d..0c5debd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
@@ -26,14 +27,26 @@ import org.jetbrains.annotations.*;
 import java.net.*;
 
 /**
- * Internal API extension for {@link org.apache.ignite.IgniteFs}.
+ * Internal API extension for {@link org.apache.ignite.IgniteFileSystem}.
  */
-public interface IgfsEx extends IgniteFs {
+public interface IgfsEx extends IgniteFileSystem {
+    /** File property: user name. */
+    public static final String PROP_USER_NAME = "usrName";
+
+    /** File property: group name. */
+    public static final String PROP_GROUP_NAME = "grpName";
+
+    /** File property: permission. */
+    public static final String PROP_PERMISSION = "permission";
+
+    /** File property: prefer writes to local node. */
+    public static final String PROP_PREFER_LOCAL_WRITES = "locWrite";
+
     /** Property name for path to Hadoop configuration. */
-    String SECONDARY_FS_CONFIG_PATH = "SECONDARY_FS_CONFIG_PATH";
+    public static final String SECONDARY_FS_CONFIG_PATH = "SECONDARY_FS_CONFIG_PATH";
 
     /** Property name for URI of file system. */
-    String SECONDARY_FS_URI = "SECONDARY_FS_URI";
+    public static final String SECONDARY_FS_URI = "SECONDARY_FS_URI";
 
     /**
      * Stops IGFS cleaning all used resources.
@@ -146,4 +159,11 @@ public interface IgfsEx extends IgniteFs {
      * @return {@code True} if proxy.
      */
     public boolean isProxy(URI path);
+
+    /**
+     * Return the given IGFS as a secondary file system.
+     *
+     * @return Secondary file system wrapper.
+     */
+    public IgfsSecondaryFileSystem asSecondary();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 43def03..e88503b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -99,13 +99,13 @@ public final class IgfsFileInfo implements Externalizable {
     }
 
     /**
-     * Constructs directory or file info with {@link org.apache.ignite.configuration.IgfsConfiguration#DFLT_BLOCK_SIZE default} block size.
+     * Constructs directory or file info with {@link org.apache.ignite.configuration.FileSystemConfiguration#DFLT_BLOCK_SIZE default} block size.
      *
      * @param isDir Constructs directory info if {@code true} or file info if {@code false}.
      * @param props Meta properties to set.
      */
     public IgfsFileInfo(boolean isDir, @Nullable Map<String, String> props) {
-        this(isDir, null, isDir ? 0 : IgfsConfiguration.DFLT_BLOCK_SIZE, 0, null, null, props, null, false,
+        this(isDir, null, isDir ? 0 : FileSystemConfiguration.DFLT_BLOCK_SIZE, 0, null, null, props, null, false,
             System.currentTimeMillis(), false);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index eff987e..614815f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -26,6 +26,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.mapreduce.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
@@ -71,7 +72,7 @@ public final class IgfsImpl implements IgfsEx {
     private IgfsDataManager data;
 
     /** FS configuration. */
-    private IgfsConfiguration cfg;
+    private FileSystemConfiguration cfg;
 
     /** IGFS context. */
     private IgfsContext igfsCtx;
@@ -89,7 +90,7 @@ public final class IgfsImpl implements IgfsEx {
     private final IgfsModeResolver modeRslvr;
 
     /** Connection to the secondary file system. */
-    private Igfs secondaryFs;
+    private IgfsSecondaryFileSystem secondaryFs;
 
     /** Busy lock. */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
@@ -124,6 +125,7 @@ public final class IgfsImpl implements IgfsEx {
      * @param igfsCtx Context.
      * @throws IgniteCheckedException In case of error.
      */
+    @SuppressWarnings("ConstantConditions")
     IgfsImpl(IgfsContext igfsCtx) throws IgniteCheckedException {
         assert igfsCtx != null;
 
@@ -376,7 +378,7 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsConfiguration configuration() {
+    @Override public FileSystemConfiguration configuration() {
         return cfg;
     }
 
@@ -938,6 +940,7 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public Collection<IgfsPath> listPaths(final IgfsPath path) {
         if (enterBusy()) {
             try {
@@ -1067,11 +1070,6 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Map<String, String> properties() {
-        return Collections.emptyMap();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgfsInputStreamAdapter open(IgfsPath path) {
         return open(path, cfg.getStreamBufferSize(), cfg.getSequentialReadsBeforePrefetch());
     }
@@ -1193,7 +1191,7 @@ public final class IgfsImpl implements IgfsEx {
 
                 IgfsMode mode = modeRslvr.resolveMode(path);
 
-                IgfsFileWorkerBatch batch = null;
+                IgfsFileWorkerBatch batch;
 
                 if (mode == PROXY)
                     throw new IgniteException("PROXY mode cannot be used in IGFS directly: " + path);
@@ -1250,6 +1248,8 @@ public final class IgfsImpl implements IgfsEx {
 
                     IgfsFileInfo oldInfo = meta.info(oldId);
 
+                    assert oldInfo != null;
+
                     if (oldInfo.isDirectory())
                         throw new IgfsPathAlreadyExistsException("Failed to create file (path points to a " +
                             "directory): " + path);
@@ -1268,7 +1268,7 @@ public final class IgfsImpl implements IgfsEx {
                 info = meta.lock(info.id());
 
                 IgfsEventAwareOutputStream os = new IgfsEventAwareOutputStream(path, info, parentId,
-                    bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, batch);
+                    bufSize == 0 ? cfg.getStreamBufferSize() : bufSize, mode, null);
 
                 if (evts.isRecordable(EVT_IGFS_FILE_OPENED_WRITE))
                     evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_OPENED_WRITE));
@@ -1339,8 +1339,7 @@ public final class IgfsImpl implements IgfsEx {
                     if (parentId == null)
                         throw new IgfsInvalidPathException("Failed to resolve parent directory: " + path);
 
-                    info = new IgfsFileInfo(cfg.getBlockSize(), /**affinity key*/null, evictExclude(path,
-                        mode == PRIMARY), props);
+                    info = new IgfsFileInfo(cfg.getBlockSize(), /**affinity key*/null, evictExclude(path, true), props);
 
                     IgniteUuid oldId = meta.putIfAbsent(parentId, path.name(), info);
 
@@ -1351,6 +1350,8 @@ public final class IgfsImpl implements IgfsEx {
                         evts.record(new IgfsEvent(path, localNode(), EVT_IGFS_FILE_CREATED));
                 }
 
+                assert info != null;
+
                 if (!info.isFile())
                     throw new IgfsInvalidPathException("Failed to open file (not a file): " + path);
 
@@ -1788,7 +1789,7 @@ public final class IgfsImpl implements IgfsEx {
 
     /**
      * Executes IGFS task with overridden maximum range length (see
-     * {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()} for more information).
+     * {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()} for more information).
      *
      * @param task Task to execute.
      * @param rslvr Optional resolver to control split boundaries.
@@ -1822,7 +1823,7 @@ public final class IgfsImpl implements IgfsEx {
 
     /**
      * Executes IGFS task asynchronously with overridden maximum range length (see
-     * {@link org.apache.ignite.configuration.IgfsConfiguration#getMaximumTaskRangeLength()} for more information).
+     * {@link org.apache.ignite.configuration.FileSystemConfiguration#getMaximumTaskRangeLength()} for more information).
      *
      * @param taskCls Task class to execute.
      * @param rslvr Optional resolver to control split boundaries.
@@ -1833,6 +1834,7 @@ public final class IgfsImpl implements IgfsEx {
      * @param arg Optional task argument.
      * @return Execution future.
      */
+    @SuppressWarnings("unchecked")
     <T, R> IgniteInternalFuture<R> executeAsync(Class<? extends IgfsTask<T, R>> taskCls,
         @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, boolean skipNonExistentFiles,
         long maxRangeLen, @Nullable T arg) {
@@ -1898,7 +1900,7 @@ public final class IgfsImpl implements IgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFs withAsync() {
+    @Override public IgniteFileSystem withAsync() {
         return new IgfsAsyncImpl(this);
     }
 
@@ -2033,7 +2035,7 @@ public final class IgfsImpl implements IgfsEx {
          * @param metrics Metrics.
          */
         IgfsEventAwareInputStream(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo,
-            int prefetchBlocks, int seqReadsBeforePrefetch, @Nullable IgfsReader secReader,
+            int prefetchBlocks, int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader,
             IgfsLocalMetrics metrics) {
             super(igfsCtx, path, fileInfo, prefetchBlocks, seqReadsBeforePrefetch, secReader, metrics);
 
@@ -2083,7 +2085,7 @@ public final class IgfsImpl implements IgfsEx {
                     private Ignite g;
 
                     @Nullable @Override public IgniteBiTuple<Long, Long> execute() {
-                        IgniteFs igfs = ((IgniteKernal)g).context().igfs().igfs(igfsName);
+                        IgniteFileSystem igfs = ((IgniteKernal)g).context().igfs().igfs(igfsName);
 
                         if (igfs == null)
                             return F.t(0L, 0L);
@@ -2129,6 +2131,7 @@ public final class IgfsImpl implements IgfsEx {
      */
     private class FormatMessageListener implements GridMessageListener {
         /** {@inheritDoc} */
+        @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
         @Override public void onMessage(UUID nodeId, Object msg) {
             if (msg instanceof IgfsDeleteMessage) {
                 ClusterNode node = igfsCtx.kernalContext().discovery().node(nodeId);
@@ -2219,4 +2222,9 @@ public final class IgfsImpl implements IgfsEx {
 
         return mode == PROXY;
     }
+
+    /** {@inheritDoc} */
+    @Override public IgfsSecondaryFileSystem asSecondary() {
+        return new IgfsSecondaryFileSystemImpl(this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
index 51e57db..fc333b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamAdapter.java
@@ -18,13 +18,15 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 
 import java.io.*;
 
 /**
  * Implementation adapter providing necessary methods.
  */
-public abstract class IgfsInputStreamAdapter extends IgfsInputStream {
+public abstract class IgfsInputStreamAdapter extends IgfsInputStream
+    implements IgfsSecondaryFileSystemPositionedReadable {
     /** {@inheritDoc} */
     @Override public long length() {
         return fileInfo().length();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
index 92b4383..5afa523 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsInputStreamImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -45,7 +46,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
 
     /** Secondary file system reader. */
     @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private final IgfsReader secReader;
+    private final IgfsSecondaryFileSystemPositionedReadable secReader;
 
     /** Logger. */
     private IgniteLogger log;
@@ -110,7 +111,7 @@ public class IgfsInputStreamImpl extends IgfsInputStreamAdapter {
      * @param metrics Local IGFS metrics.
      */
     IgfsInputStreamImpl(IgfsContext igfsCtx, IgfsPath path, IgfsFileInfo fileInfo, int prefetchBlocks,
-        int seqReadsBeforePrefetch, @Nullable IgfsReader secReader, IgfsLocalMetrics metrics) {
+        int seqReadsBeforePrefetch, @Nullable IgfsSecondaryFileSystemPositionedReadable secReader, IgfsLocalMetrics metrics) {
         assert igfsCtx != null;
         assert path != null;
         assert fileInfo != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
index f6b17bf..fa90e21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsJobImpl.java
@@ -79,7 +79,7 @@ public class IgfsJobImpl implements ComputeJob, GridInternalWrapper<IgfsJob> {
 
     /** {@inheritDoc} */
     @Override public Object execute() {
-        IgniteFs fs = ignite.fileSystem(igfsName);
+        IgniteFileSystem fs = ignite.fileSystem(igfsName);
 
         try (IgfsInputStream in = fs.open(path)) {
             IgfsFileRange split = new IgfsFileRange(path, start, len);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index 9d1795a..adc0254 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -51,7 +52,7 @@ import static org.apache.ignite.transactions.TransactionIsolation.*;
 @SuppressWarnings("all")
 public class IgfsMetaManager extends IgfsManager {
     /** IGFS configuration. */
-    private IgfsConfiguration cfg;
+    private FileSystemConfiguration cfg;
 
     /** Metadata cache. */
     private GridCache<Object, Object> metaCache;
@@ -1588,7 +1589,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Output stream descriptor.
      * @throws IgniteCheckedException If file creation failed.
      */
-    public IgfsSecondaryOutputStreamDescriptor createDual(final Igfs fs,
+    public IgfsSecondaryOutputStreamDescriptor createDual(final IgfsSecondaryFileSystem fs,
         final IgfsPath path,
         final boolean simpleCreate,
         @Nullable final Map<String, String> props,
@@ -1752,7 +1753,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Output stream descriptor.
      * @throws IgniteCheckedException If output stream open for append has failed.
      */
-    public IgfsSecondaryOutputStreamDescriptor appendDual(final Igfs fs, final IgfsPath path,
+    public IgfsSecondaryOutputStreamDescriptor appendDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
         final int bufSize) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -1783,7 +1784,7 @@ public class IgfsMetaManager extends IgfsManager {
                             if (remainder > 0) {
                                 int blockIdx = (int)(len / blockSize);
 
-                                IgfsReader reader = fs.open(path, bufSize);
+                                IgfsSecondaryFileSystemPositionedReadable reader = fs.open(path, bufSize);
 
                                 try {
                                     igfsCtx.data().dataBlock(info, path, blockIdx, reader).get();
@@ -1832,7 +1833,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Input stream descriptor.
      * @throws IgniteCheckedException If input stream open has failed.
      */
-    public IgfsSecondaryInputStreamDescriptor openDual(final Igfs fs, final IgfsPath path,
+    public IgfsSecondaryInputStreamDescriptor openDual(final IgfsSecondaryFileSystem fs, final IgfsPath path,
         final int bufSize)
         throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
@@ -1893,7 +1894,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info or {@code null} if file not found.
      * @throws IgniteCheckedException If sync task failed.
      */
-    @Nullable public IgfsFileInfo synchronizeFileDual(final Igfs fs, final IgfsPath path)
+    @Nullable public IgfsFileInfo synchronizeFileDual(final IgfsSecondaryFileSystem fs, final IgfsPath path)
         throws IgniteCheckedException {
         assert fs != null;
         assert path != null;
@@ -1941,7 +1942,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return {@code True} in case rename was successful.
      * @throws IgniteCheckedException If directory creation failed.
      */
-    public boolean mkdirsDual(final Igfs fs, final IgfsPath path, final Map<String, String> props)
+    public boolean mkdirsDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final Map<String, String> props)
         throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -2025,7 +2026,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Operation result.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean renameDual(final Igfs fs, final IgfsPath src, final IgfsPath dest) throws
+    public boolean renameDual(final IgfsSecondaryFileSystem fs, final IgfsPath src, final IgfsPath dest) throws
         IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -2124,7 +2125,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Operation result.
      * @throws IgniteCheckedException If delete failed.
      */
-    public boolean deleteDual(final Igfs fs, final IgfsPath path, final boolean recursive)
+    public boolean deleteDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final boolean recursive)
         throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
@@ -2190,7 +2191,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return Update file info.
      * @throws IgniteCheckedException If update failed.
      */
-    public IgfsFileInfo updateDual(final Igfs fs, final IgfsPath path, final Map<String, String> props)
+    public IgfsFileInfo updateDual(final IgfsSecondaryFileSystem fs, final IgfsPath path, final Map<String, String> props)
         throws IgniteCheckedException {
         assert fs != null;
         assert path != null;
@@ -2243,7 +2244,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @return File info of the end path.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsFileInfo synchronize(Igfs fs,
+    private IgfsFileInfo synchronize(IgfsSecondaryFileSystem fs,
         IgfsPath startPath,
         IgfsFileInfo startPathInfo,
         IgfsPath endPath,
@@ -2328,7 +2329,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     private <T> T synchronizeAndExecute(SynchronizationTask<T> task,
-        Igfs fs,
+        IgfsSecondaryFileSystem fs,
         boolean strict,
         IgfsPath... paths)
         throws IgniteCheckedException
@@ -2349,7 +2350,7 @@ public class IgfsMetaManager extends IgfsManager {
      * @throws IgniteCheckedException If failed.
      */
     private <T> T synchronizeAndExecute(SynchronizationTask<T> task,
-        Igfs fs,
+        IgfsSecondaryFileSystem fs,
         boolean strict,
         @Nullable Collection<IgniteUuid> extraLockIds,
         IgfsPath... paths)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNoopProcessor.java
index a299fc4..41dcc31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsNoopProcessor.java
@@ -49,12 +49,12 @@ public class IgfsNoopProcessor extends IgfsProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFs> igfss() {
+    @Override public Collection<IgniteFileSystem> igfss() {
         return Collections.emptyList();
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFs igfs(@Nullable String name) {
+    @Nullable @Override public IgniteFileSystem igfs(@Nullable String name) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
index 5941579..6c6dd9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsOutputStreamImpl.java
@@ -405,7 +405,7 @@ class IgfsOutputStreamImpl extends IgfsOutputStreamAdapter {
         if (!igfsCtx.configuration().isFragmentizerEnabled())
             return null;
 
-        if (!Boolean.parseBoolean(fileInfo.properties().get(IgniteFs.PROP_PREFER_LOCAL_WRITES)))
+        if (!Boolean.parseBoolean(fileInfo.properties().get(IgfsEx.PROP_PREFER_LOCAL_WRITES)))
             return null;
 
         int blockSize = fileInfo.blockSize();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
index 847cd50..e1b5114 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessor.java
@@ -51,8 +51,8 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
     private static final String NULL_NAME = UUID.randomUUID().toString();
 
     /** Converts context to IGFS. */
-    private static final IgniteClosure<IgfsContext,IgniteFs> CTX_TO_IGFS = new C1<IgfsContext, IgniteFs>() {
-        @Override public IgniteFs apply(IgfsContext igfsCtx) {
+    private static final IgniteClosure<IgfsContext,IgniteFileSystem> CTX_TO_IGFS = new C1<IgfsContext, IgniteFileSystem>() {
+        @Override public IgniteFileSystem apply(IgfsContext igfsCtx) {
             return igfsCtx.igfs();
         }
     };
@@ -73,17 +73,17 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         if (ctx.config().isDaemon())
             return;
 
-        IgfsConfiguration[] cfgs = ctx.config().getIgfsConfiguration();
+        FileSystemConfiguration[] cfgs = ctx.config().getFileSystemConfiguration();
 
         assert cfgs != null && cfgs.length > 0;
 
         validateLocalIgfsConfigurations(cfgs);
 
         // Start IGFS instances.
-        for (IgfsConfiguration cfg : cfgs) {
+        for (FileSystemConfiguration cfg : cfgs) {
             IgfsContext igfsCtx = new IgfsContext(
                 ctx,
-                new IgfsConfiguration(cfg),
+                new FileSystemConfiguration(cfg),
                 new IgfsMetaManager(),
                 new IgfsDataManager(),
                 new IgfsServerManager(),
@@ -167,12 +167,12 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public Collection<IgniteFs> igfss() {
+    @Override public Collection<IgniteFileSystem> igfss() {
         return F.viewReadOnly(igfsCache.values(), CTX_TO_IGFS);
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public IgniteFs igfs(@Nullable String name) {
+    @Override @Nullable public IgniteFileSystem igfs(@Nullable String name) {
         IgfsContext igfsCtx = igfsCache.get(maskName(name));
 
         return igfsCtx == null ? null : igfsCtx.igfs();
@@ -202,7 +202,7 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
         // is daemon;
         // doesn't have configured IGFS;
         // doesn't have configured caches.
-        if (gridCfg.isDaemon() || F.isEmpty(gridCfg.getIgfsConfiguration()) ||
+        if (gridCfg.isDaemon() || F.isEmpty(gridCfg.getFileSystemConfiguration()) ||
             F.isEmpty(gridCfg.getCacheConfiguration()))
             return;
 
@@ -216,9 +216,9 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
 
         Collection<IgfsAttributes> attrVals = new ArrayList<>();
 
-        assert gridCfg.getIgfsConfiguration() != null;
+        assert gridCfg.getFileSystemConfiguration() != null;
 
-        for (IgfsConfiguration igfsCfg : gridCfg.getIgfsConfiguration()) {
+        for (FileSystemConfiguration igfsCfg : gridCfg.getFileSystemConfiguration()) {
             CacheConfiguration cacheCfg = cacheCfgs.get(igfsCfg.getDataCacheName());
 
             if (cacheCfg == null)
@@ -258,10 +258,10 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
      * @param cfgs IGFS configurations
      * @throws IgniteCheckedException If any of IGFS configurations is invalid.
      */
-    private void validateLocalIgfsConfigurations(IgfsConfiguration[] cfgs) throws IgniteCheckedException {
+    private void validateLocalIgfsConfigurations(FileSystemConfiguration[] cfgs) throws IgniteCheckedException {
         Collection<String> cfgNames = new HashSet<>();
 
-        for (IgfsConfiguration cfg : cfgs) {
+        for (FileSystemConfiguration cfg : cfgs) {
             String name = cfg.getName();
 
             if (cfgNames.contains(name))
@@ -333,7 +333,7 @@ public class IgfsProcessor extends IgfsProcessorAdapter {
             if (secondary) {
                 // When working in any mode except of primary, secondary FS config must be provided.
                 assertParameter(cfg.getSecondaryFileSystem() != null,
-                    "secondaryFileSystem cannot be null when mode is SECONDARY");
+                    "secondaryFileSystem cannot be null when mode is not " + IgfsMode.PRIMARY);
             }
 
             cfgNames.add(name);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorAdapter.java
index b695104..ee38ab3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorAdapter.java
@@ -46,7 +46,7 @@ public abstract class IgfsProcessorAdapter extends GridProcessorAdapter {
      *
      * @return Collection of IGFS instances.
      */
-    public abstract Collection<IgniteFs> igfss();
+    public abstract Collection<IgniteFileSystem> igfss();
 
     /**
      * Gets IGFS instance.
@@ -54,7 +54,7 @@ public abstract class IgfsProcessorAdapter extends GridProcessorAdapter {
      * @param name (Nullable) IGFS name.
      * @return IGFS instance.
      */
-    @Nullable public abstract IgniteFs igfs(@Nullable String name);
+    @Nullable public abstract IgniteFileSystem igfs(@Nullable String name);
 
     /**
      * Gets server endpoints for particular IGFS.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
new file mode 100644
index 0000000..683b317
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
@@ -0,0 +1,121 @@
+/*
+ * 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.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Secondary file system over native IGFS.
+ */
+class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
+    /** Delegate. */
+    private final IgfsImpl igfs;
+
+    /**
+     * Constructor.
+     *
+     * @param igfs Delegate.
+     */
+    IgfsSecondaryFileSystemImpl(IgfsImpl igfs) {
+        this.igfs = igfs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean exists(IgfsPath path) {
+        return igfs.exists(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteException {
+        return igfs.update(path, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void rename(IgfsPath src, IgfsPath dest) throws IgniteException {
+        igfs.rename(src, dest);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(IgfsPath path, boolean recursive) throws IgniteException {
+        return igfs.delete(path, recursive);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(IgfsPath path) throws IgniteException {
+        igfs.mkdirs(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) throws IgniteException {
+        igfs.mkdirs(path, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteException {
+        return igfs.listPaths(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteException {
+        return igfs.listFiles(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize)
+        throws IgniteException {
+        return igfs.open(path, bufSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream create(IgfsPath path, boolean overwrite) throws IgniteException {
+        return igfs.create(path, overwrite);
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
+        long blockSize, @Nullable Map<String, String> props) throws IgniteException {
+        return igfs.create(path, bufSize, overwrite, replication, blockSize, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
+        @Nullable Map<String, String> props) throws IgniteException {
+        return igfs.append(path, bufSize, create, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(IgfsPath path) throws IgniteException {
+        return igfs.info(path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long usedSpaceSize() throws IgniteException {
+        return igfs.usedSpaceSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, String> properties() {
+        return Collections.emptyMap();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
index 6e48103..e9ba6f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryInputStreamDescriptor.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
-import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 
 /**
  * Descriptor of an input stream opened to the secondary file system.
@@ -27,7 +27,7 @@ public class IgfsSecondaryInputStreamDescriptor {
     private final IgfsFileInfo info;
 
     /** Secondary file system input stream wrapper. */
-    private final IgfsReader secReader;
+    private final IgfsSecondaryFileSystemPositionedReadable secReader;
 
     /**
      * Constructor.
@@ -35,7 +35,7 @@ public class IgfsSecondaryInputStreamDescriptor {
      * @param info File info in the primary file system.
      * @param secReader Secondary file system reader.
      */
-    IgfsSecondaryInputStreamDescriptor(IgfsFileInfo info, IgfsReader secReader) {
+    IgfsSecondaryInputStreamDescriptor(IgfsFileInfo info, IgfsSecondaryFileSystemPositionedReadable secReader) {
         assert info != null;
         assert secReader != null;
 
@@ -53,7 +53,7 @@ public class IgfsSecondaryInputStreamDescriptor {
     /**
      * @return Secondary file system reader.
      */
-    IgfsReader reader() {
+    IgfsSecondaryFileSystemPositionedReadable reader() {
         return secReader;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
index e1f4a0f..643eeff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServerManager.java
@@ -29,7 +29,7 @@ import org.apache.ignite.thread.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
 
 /**
  * IGFS server manager.
@@ -49,7 +49,7 @@ public class IgfsServerManager extends IgfsManager {
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
-        IgfsConfiguration igfsCfg = igfsCtx.configuration();
+        FileSystemConfiguration igfsCfg = igfsCtx.configuration();
         Map<String,String> cfg = igfsCfg.getIpcEndpointConfiguration();
 
         if (F.isEmpty(cfg)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfs.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfs.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfs.java
index 2258d4d..edcde6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfs.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfs.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import java.io.*;
 
 /**
- * Data transfer object for {@link org.apache.ignite.IgniteFs}.
+ * Data transfer object for {@link org.apache.ignite.IgniteFileSystem}.
  */
 public class VisorIgfs implements Serializable {
     /** */
@@ -66,7 +66,7 @@ public class VisorIgfs implements Serializable {
      * @param igfs Source IGFS.
      * @return Data transfer object for given IGFS.
      */
-    public static VisorIgfs from(IgniteFs igfs) {
+    public static VisorIgfs from(IgniteFileSystem igfs) {
         assert igfs != null;
 
         return new VisorIgfs(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfsProfilerClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfsProfilerClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfsProfilerClearTask.java
index f63e669..814fc50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfsProfilerClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/igfs/VisorIgfsProfilerClearTask.java
@@ -59,7 +59,7 @@ public class VisorIgfsProfilerClearTask extends VisorOneNodeTask<String, IgniteB
             int notDeleted = 0;
 
             try {
-                IgniteFs igfs = ignite.fileSystem(arg);
+                IgniteFileSystem igfs = ignite.fileSystem(arg);
 
                 Path logsDir = resolveIgfsProfilerLogsDir(igfs);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
index 96c69d9..cc6ae63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
@@ -114,7 +114,7 @@ public class VisorGridConfiguration implements Serializable {
         rest = VisorRestConfiguration.from(c);
         userAttrs = c.getUserAttributes();
         caches = VisorCacheConfiguration.list(ignite, c.getCacheConfiguration());
-        igfss = VisorIgfsConfiguration.list(c.getIgfsConfiguration());
+        igfss = VisorIgfsConfiguration.list(c.getFileSystemConfiguration());
         streamers = VisorStreamerConfiguration.list(c.getStreamerConfiguration());
         env = new HashMap<>(getenv());
         sysProps = getProperties();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
index 78943a2..9064ea6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.visor.node;
 
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
@@ -120,7 +121,7 @@ public class VisorIgfsConfiguration implements Serializable {
      * @param igfs IGFS configuration.
      * @return Data transfer object for IGFS configuration properties.
      */
-    public static VisorIgfsConfiguration from(IgfsConfiguration igfs) {
+    public static VisorIgfsConfiguration from(FileSystemConfiguration igfs) {
         VisorIgfsConfiguration cfg = new VisorIgfsConfiguration();
 
         cfg.name = igfs.getName();
@@ -132,7 +133,7 @@ public class VisorIgfsConfiguration implements Serializable {
         cfg.perNodeBatchSize = igfs.getPerNodeBatchSize();
         cfg.perNodeParallelBatchCnt = igfs.getPerNodeParallelBatchCount();
 
-        Igfs secFs = igfs.getSecondaryFileSystem();
+        IgfsSecondaryFileSystem secFs = igfs.getSecondaryFileSystem();
 
         if (secFs != null) {
             Map<String, String> props = secFs.properties();
@@ -171,13 +172,13 @@ public class VisorIgfsConfiguration implements Serializable {
      * @param igfss Igfs configurations.
      * @return igfs configurations properties.
      */
-    public static Iterable<VisorIgfsConfiguration> list(IgfsConfiguration[] igfss) {
+    public static Iterable<VisorIgfsConfiguration> list(FileSystemConfiguration[] igfss) {
         if (igfss == null)
             return Collections.emptyList();
 
         final Collection<VisorIgfsConfiguration> cfgs = new ArrayList<>(igfss.length);
 
-        for (IgfsConfiguration igfs : igfss)
+        for (FileSystemConfiguration igfs : igfss)
             cfgs.add(from(igfs));
 
         return cfgs;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index 43a987c..b86ea5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -151,7 +151,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
         try {
             IgfsProcessorAdapter igfsProc = ((IgniteKernal)ignite).context().igfs();
 
-            for (IgniteFs igfs : igfsProc.igfss()) {
+            for (IgniteFileSystem igfs : igfsProc.igfss()) {
                 long start0 = U.currentTimeMillis();
 
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index b03674f..51e3c6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.*;
 import java.util.zip.*;
 
 import static java.lang.System.*;
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
+import static org.apache.ignite.configuration.FileSystemConfiguration.*;
 import static org.apache.ignite.events.EventType.*;
 
 /**
@@ -592,7 +592,7 @@ public class VisorTaskUtils {
      * @return {@link Path} to log dir or {@code null} if not found.
      * @throws IgniteCheckedException if failed to resolve.
      */
-    public static Path resolveIgfsProfilerLogsDir(IgniteFs igfs) throws IgniteCheckedException {
+    public static Path resolveIgfsProfilerLogsDir(IgniteFileSystem igfs) throws IgniteCheckedException {
         String logsDir;
 
         if (igfs instanceof IgfsEx)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml b/modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml
index 2dc952a..43f9395 100644
--- a/modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml
+++ b/modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml
@@ -29,12 +29,12 @@
 
     <property>
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <property>
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/hadoop/core-site-loopback.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/hadoop/core-site-loopback.xml b/modules/core/src/test/config/hadoop/core-site-loopback.xml
index 63902fa..e1d1320 100644
--- a/modules/core/src/test/config/hadoop/core-site-loopback.xml
+++ b/modules/core/src/test/config/hadoop/core-site-loopback.xml
@@ -29,12 +29,12 @@
 
     <property>
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <property>
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/hadoop/core-site-secondary.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/hadoop/core-site-secondary.xml b/modules/core/src/test/config/hadoop/core-site-secondary.xml
index 648df36..fa301be 100644
--- a/modules/core/src/test/config/hadoop/core-site-secondary.xml
+++ b/modules/core/src/test/config/hadoop/core-site-secondary.xml
@@ -29,12 +29,12 @@
 
     <property>
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <property>
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 
     <property>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/hadoop/core-site.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/hadoop/core-site.xml b/modules/core/src/test/config/hadoop/core-site.xml
index eb16ce7..0a9eee7 100644
--- a/modules/core/src/test/config/hadoop/core-site.xml
+++ b/modules/core/src/test/config/hadoop/core-site.xml
@@ -29,11 +29,11 @@
 
     <property>
         <name>fs.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v1.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem</value>
     </property>
 
     <property>
         <name>fs.AbstractFileSystem.igfs.impl</name>
-        <value>org.apache.ignite.igfs.hadoop.v2.IgfsHadoopFileSystem</value>
+        <value>org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem</value>
     </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/igfs-loopback.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/igfs-loopback.xml b/modules/core/src/test/config/igfs-loopback.xml
index 81ce501..46377d9 100644
--- a/modules/core/src/test/config/igfs-loopback.xml
+++ b/modules/core/src/test/config/igfs-loopback.xml
@@ -78,9 +78,9 @@
             </bean>
         </property>
 
-        <property name="igfsConfiguration">
+        <property name="fileSystemConfiguration">
             <list>
-                <bean class="org.apache.ignite.configuration.IgfsConfiguration">
+                <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
                     <property name="metaCacheName" value="igfs-meta"/>
                     <property name="dataCacheName" value="igfs-data"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/config/igfs-shmem.xml
----------------------------------------------------------------------
diff --git a/modules/core/src/test/config/igfs-shmem.xml b/modules/core/src/test/config/igfs-shmem.xml
index d30051b..c08f78d 100644
--- a/modules/core/src/test/config/igfs-shmem.xml
+++ b/modules/core/src/test/config/igfs-shmem.xml
@@ -78,9 +78,9 @@
             </bean>
         </property>
 
-        <property name="igfsConfiguration">
+        <property name="fileSystemConfiguration">
             <list>
-                <bean class="org.apache.ignite.configuration.IgfsConfiguration">
+                <bean class="org.apache.ignite.configuration.FileSystemConfiguration">
                     <property name="name" value="igfs"/>
                     <property name="metaCacheName" value="igfs-meta"/>
                     <property name="dataCacheName" value="igfs-data"/>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
index 364eaa1..2d9d269 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsEventsAbstractSelfTest.java
@@ -40,7 +40,7 @@ import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.testframework.GridTestUtils.*;
 
 /**
- * Tests events, generated by {@link org.apache.ignite.IgniteFs} implementation.
+ * Tests events, generated by {@link org.apache.ignite.IgniteFileSystem} implementation.
  */
 public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest {
     /** IGFS. */
@@ -84,8 +84,8 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
     /**
      * @return IGFS configuration for this test.
      */
-    protected IgfsConfiguration getIgfsConfiguration() throws IgniteCheckedException {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+    protected FileSystemConfiguration getIgfsConfiguration() throws IgniteCheckedException {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
         igfsCfg.setMetaCacheName("metaCache");
@@ -109,7 +109,7 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
      * @return Grid configuration.
      * @throws Exception If failed.
      */
-    protected IgniteConfiguration getConfiguration(String gridName, IgfsConfiguration igfsCfg) throws Exception {
+    protected IgniteConfiguration getConfiguration(String gridName, FileSystemConfiguration igfsCfg) throws Exception {
         IgniteConfiguration cfg = IgnitionEx.loadConfiguration("config/hadoop/default-config.xml").get1();
 
         assert cfg != null;
@@ -118,7 +118,7 @@ public abstract class IgfsEventsAbstractSelfTest extends GridCommonAbstractTest
 
         cfg.setIncludeEventTypes(concat(EVTS_IGFS, EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED));
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         cfg.setCacheConfiguration(getCacheConfiguration(gridName));
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
index 6ae9c86..3fa0804 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerAbstractSelfTest.java
@@ -63,7 +63,7 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
 
         cfg.setCacheConfiguration(metaConfiguration(), dataConfiguration());
 
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setName("igfs");
         igfsCfg.setMetaCacheName(META_CACHE_NAME);
@@ -76,7 +76,7 @@ public class IgfsFragmentizerAbstractSelfTest extends IgfsCommonAbstractTest {
         igfsCfg.setFragmentizerThrottlingBlockLength(16 * IGFS_BLOCK_SIZE);
         igfsCfg.setFragmentizerThrottlingDelay(10);
 
-        cfg.setIgfsConfiguration(igfsCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
index 99a40c4..b51db0c 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
@@ -36,7 +36,7 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testReadFragmentizing() throws Exception {
-        IgniteFs igfs = grid(0).fileSystem("igfs");
+        IgniteFileSystem igfs = grid(0).fileSystem("igfs");
 
         IgfsPath path = new IgfsPath("/someFile");
 
@@ -108,7 +108,7 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
         int fileSize = 30 * IGFS_GROUP_SIZE * IGFS_BLOCK_SIZE;
 
         while (written < fileSize) {
-            IgniteFs igfs = grid(igfsIdx).fileSystem("igfs");
+            IgniteFileSystem igfs = grid(igfsIdx).fileSystem("igfs");
 
             try (IgfsOutputStream out = igfs.append(path, true)) {
                 byte[] data = new byte[chunkSize];
@@ -131,7 +131,7 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
             }
         }
 
-        IgniteFs igfs = grid(0).fileSystem("igfs");
+        IgniteFileSystem igfs = grid(0).fileSystem("igfs");
 
         try (IgfsInputStream in = igfs.open(path)) {
             i = 0;
@@ -180,7 +180,7 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
 
         int fileSize = 50 * IGFS_GROUP_SIZE * IGFS_BLOCK_SIZE;
 
-        IgniteFs igfs = grid(0).fileSystem("igfs");
+        IgniteFileSystem igfs = grid(0).fileSystem("igfs");
 
         byte[] chunk = new byte[chunkSize];
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6423cf02/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerTopologySelfTest.java
index 59faa42..cb7b63a 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerTopologySelfTest.java
@@ -33,7 +33,7 @@ public class IgfsFragmentizerTopologySelfTest extends IgfsFragmentizerAbstractSe
         try {
             IgfsPath path = new IgfsPath("/someFile");
 
-            IgniteFs igfs = grid(1).fileSystem("igfs");
+            IgniteFileSystem igfs = grid(1).fileSystem("igfs");
 
             try (IgfsOutputStream out = igfs.create(path, true)) {
                 for (int i = 0; i < 10 * IGFS_GROUP_SIZE; i++)