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/03/03 14:08:17 UTC

[01/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-386 7d46deb24 -> e1ffc10f6


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
index 5fad86f..1688dec 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
@@ -40,7 +40,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suite() throws Exception {
-        GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
+        HadoopClassLoader ldr = new HadoopClassLoader(null);
 
         TestSuite suite = new TestSuite("Ignite FS Events Test Suite");
 
@@ -60,7 +60,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suiteNoarchOnly() throws Exception {
-        GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
+        HadoopClassLoader ldr = new HadoopClassLoader(null);
 
         TestSuite suite = new TestSuite("Ignite IGFS Events Test Suite Noarch Only");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoaderTest.java
deleted file mode 100644
index 767be7c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoaderTest.java
+++ /dev/null
@@ -1,69 +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 junit.framework.*;
-import org.apache.hadoop.mapreduce.*;
-
-/**
- *
- */
-public class GridHadoopClassLoaderTest extends TestCase {
-    /** */
-    GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClassLoading() throws Exception {
-        assertNotSame(Test1.class, ldr.loadClass(Test1.class.getName()));
-        assertNotSame(Test2.class, ldr.loadClass(Test2.class.getName()));
-        assertSame(Test3.class, ldr.loadClass(Test3.class.getName()));
-    }
-
-//    public void testDependencySearch() {
-//        assertTrue(ldr.hasExternalDependencies(Test1.class.getName(), new HashSet<String>()));
-//        assertTrue(ldr.hasExternalDependencies(Test2.class.getName(), new HashSet<String>()));
-//    }
-
-    /**
-     *
-     */
-    private static class Test1 {
-        /** */
-        Test2 t2;
-
-        /** */
-        Job[][] jobs = new Job[4][4];
-    }
-
-    /**
-     *
-     */
-    private static abstract class Test2 {
-        /** */
-        abstract Test1 t1();
-    }
-
-    /**
-     *
-     */
-    private static class Test3 {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
index 80cd226..7e2b2de 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
@@ -169,7 +169,7 @@ public class GridHadoopCommandLineTest extends GridCommonAbstractTest {
                 if (line.startsWith("</configuration>"))
                     out.println(
                         "    <property>\n" +
-                        "        <name>" + GridHadoopUtils.JOB_COUNTER_WRITER_PROPERTY + "</name>\n" +
+                        "        <name>" + HadoopUtils.JOB_COUNTER_WRITER_PROPERTY + "</name>\n" +
                         "        <value>" + GridHadoopFSCounterWriter.class.getName() + "</value>\n" +
                         "    </property>\n");
 
@@ -210,7 +210,7 @@ public class GridHadoopCommandLineTest extends GridCommonAbstractTest {
         String sep = ":";
 
         String ggClsPath = GridHadoopJob.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
-            GridHadoopJobTracker.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
+            HadoopJobTracker.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
             ConcurrentHashMap8.class.getProtectionDomain().getCodeSource().getLocation().getPath();
 
         ProcessBuilder res = new ProcessBuilder();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
index b1b0275..947159c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
@@ -611,7 +611,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
 
         /** {@inheritDoc} */
         @Override public GridHadoopJobInfo info() {
-            return new GridHadoopDefaultJobInfo() {
+            return new HadoopDefaultJobInfo() {
                 @Override public int reducers() {
                     return reducers;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java
index 49099fc..c9c577d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopGroupingTest.java
@@ -26,7 +26,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Grouping test.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
index 3aa74d0..7dffbc3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
@@ -30,7 +30,7 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Job tracker self test.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
index dda041c..89318f3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.ignite.internal.processors.hadoop.examples.*;
 
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Tests map-reduce execution with embedded mode.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
index 072e764..7b756da 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
@@ -35,7 +35,7 @@ import org.apache.ignite.testframework.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Test of whole cycle of map-reduce processing via Job tracker.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java
index 3a2c397..9f107d1 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSortingTest.java
@@ -29,7 +29,7 @@ import java.io.*;
 import java.net.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Tests correct sorting.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
index 0b15a2c..c086719 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
@@ -39,7 +39,7 @@ public class GridHadoopSplitWrapperSelfTest extends GridHadoopAbstractSelfTest {
 
         assertEquals("/path/to/file:100+500", nativeSplit.toString());
 
-        GridHadoopSplitWrapper split = GridHadoopUtils.wrapSplit(10, nativeSplit, nativeSplit.getLocations());
+        GridHadoopSplitWrapper split = HadoopUtils.wrapSplit(10, nativeSplit, nativeSplit.getLocations());
 
         assertEquals("[host1, host2]", Arrays.toString(split.hosts()));
 
@@ -53,7 +53,7 @@ public class GridHadoopSplitWrapperSelfTest extends GridHadoopAbstractSelfTest {
 
         final GridHadoopSplitWrapper res = (GridHadoopSplitWrapper)in.readObject();
 
-        assertEquals("/path/to/file:100+500", GridHadoopUtils.unwrapSplit(res).toString());
+        assertEquals("/path/to/file:100+500", HadoopUtils.unwrapSplit(res).toString());
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
             @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
index 06fffa2..541ed86 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
@@ -39,7 +39,7 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Tests map-reduce task execution basics.
@@ -425,7 +425,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
             try {
                 super.run(ctx);
             }
-            catch (GridHadoopTaskCancelledException e) {
+            catch (HadoopTaskCancelledException e) {
                 cancelledTasks.incrementAndGet();
 
                 throw e;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
index 15ac125..c6b10bd 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
@@ -24,7 +24,7 @@ import org.apache.ignite.internal.processors.hadoop.v2.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Tests of Map, Combine and Reduce task executions via running of job of hadoop API v1.
@@ -43,7 +43,7 @@ public class GridHadoopTasksV1Test extends GridHadoopTasksAllVersionsTest {
 
         setupFileSystems(jobConf);
 
-        GridHadoopDefaultJobInfo jobInfo = createJobInfo(jobConf);
+        HadoopDefaultJobInfo jobInfo = createJobInfo(jobConf);
 
         GridHadoopJobId jobId = new GridHadoopJobId(new UUID(0, 0), 0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
index e48eb01..13dd688 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
@@ -28,7 +28,7 @@ import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Tests of Map, Combine and Reduce task executions via running of job of hadoop API v2.
@@ -61,7 +61,7 @@ public class GridHadoopTasksV2Test extends GridHadoopTasksAllVersionsTest {
 
         Job hadoopJob = GridHadoopWordCount2.getJob(inFile, outFile);
 
-        GridHadoopDefaultJobInfo jobInfo = createJobInfo(hadoopJob.getConfiguration());
+        HadoopDefaultJobInfo jobInfo = createJobInfo(hadoopJob.getConfiguration());
 
         GridHadoopJobId jobId = new GridHadoopJobId(new UUID(0, 0), 0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
index 4e0aa9b..80b00a6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
@@ -192,7 +192,7 @@ class GridHadoopTestTaskContext extends GridHadoopV2TaskContext {
     private static DataInput jobConfDataInput(GridHadoopJob job) throws IgniteCheckedException {
         JobConf jobConf = new JobConf();
 
-        for (Map.Entry<String, String> e : ((GridHadoopDefaultJobInfo)job.info()).properties().entrySet())
+        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)job.info()).properties().entrySet())
             jobConf.set(e.getKey(), e.getValue());
 
         ByteArrayOutputStream buf = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
index b201614..c7a456b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.processors.hadoop.v2.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Self test of {@link GridHadoopV2Job}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
new file mode 100644
index 0000000..a3289cb
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoaderTest.java
@@ -0,0 +1,69 @@
+/*
+ * 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 junit.framework.*;
+import org.apache.hadoop.mapreduce.*;
+
+/**
+ *
+ */
+public class HadoopClassLoaderTest extends TestCase {
+    /** */
+    HadoopClassLoader ldr = new HadoopClassLoader(null);
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClassLoading() throws Exception {
+        assertNotSame(Test1.class, ldr.loadClass(Test1.class.getName()));
+        assertNotSame(Test2.class, ldr.loadClass(Test2.class.getName()));
+        assertSame(Test3.class, ldr.loadClass(Test3.class.getName()));
+    }
+
+//    public void testDependencySearch() {
+//        assertTrue(ldr.hasExternalDependencies(Test1.class.getName(), new HashSet<String>()));
+//        assertTrue(ldr.hasExternalDependencies(Test2.class.getName(), new HashSet<String>()));
+//    }
+
+    /**
+     *
+     */
+    private static class Test1 {
+        /** */
+        Test2 t2;
+
+        /** */
+        Job[][] jobs = new Job[4][4];
+    }
+
+    /**
+     *
+     */
+    private static abstract class Test2 {
+        /** */
+        abstract Test1 t1();
+    }
+
+    /**
+     *
+     */
+    private static class Test3 {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
index 52bfa98..ab65e77 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
@@ -32,7 +32,7 @@ import org.apache.ignite.internal.util.typedef.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Job tracker self test.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 5398d91..f599ed9 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -50,7 +50,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         downloadHadoop();
         downloadHive();
 
-        GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
+        HadoopClassLoader ldr = new HadoopClassLoader(null);
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
index 477dd84..1f428e4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
@@ -36,7 +36,7 @@ public class IgniteIgfsLinuxAndMacOSTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         downloadHadoop();
 
-        GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
+        HadoopClassLoader ldr = new HadoopClassLoader(null);
 
         TestSuite suite = new TestSuite("Ignite IGFS Test Suite For Linux And Mac OS");
 


[29/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
# IGNITE-386: WIP on internal namings (4).


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

Branch: refs/heads/ignite-386
Commit: 17c8d0d90fe092a7454084126b7054f9078e8933
Parents: 1c4b00d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 3 16:05:27 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 3 16:05:27 2015 +0300

----------------------------------------------------------------------
 .../fs/IgniteHadoopSecondaryFileSystem.java     |   10 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   24 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   20 +-
 .../ignite/internal/igfs/hadoop/HadoopIgfs.java |  198 ++
 .../HadoopIgfsCommunicationException.java       |   57 +
 .../igfs/hadoop/HadoopIgfsEndpoint.java         |  210 ++
 .../internal/igfs/hadoop/HadoopIgfsEx.java      |   88 +
 .../internal/igfs/hadoop/HadoopIgfsFuture.java  |   94 +
 .../internal/igfs/hadoop/HadoopIgfsInProc.java  |  409 ++++
 .../internal/igfs/hadoop/HadoopIgfsIo.java      |   76 +
 .../igfs/hadoop/HadoopIgfsIpcIoListener.java    |   36 +
 .../igfs/hadoop/HadoopIgfsJclLogger.java        |  112 +
 .../igfs/hadoop/HadoopIgfsProperties.java       |   88 +
 .../igfs/hadoop/HadoopIgfsProxyInputStream.java |  335 +++
 .../hadoop/HadoopIgfsProxyOutputStream.java     |  165 ++
 .../internal/igfs/hadoop/HadoopIgfsReader.java  |  104 +
 .../igfs/hadoop/HadoopIgfsStreamDelegate.java   |   96 +
 .../hadoop/HadoopIgfsStreamEventListener.java   |   39 +
 .../internal/igfs/hadoop/HadoopIgfsUtils.java   |  131 ++
 .../internal/igfs/hadoop/HadoopIgfsWrapper.java |  511 +++++
 .../igfs/hadoop/HadoopInputIgfsStream.java      |  626 ++++++
 .../internal/igfs/hadoop/HadoopIpcIgfsIo.java   |  599 ++++++
 .../internal/igfs/hadoop/HadoopOutProcIgfs.java |  466 +++++
 .../igfs/hadoop/HadoopOutputIgfsStream.java     |  201 ++
 .../ignite/internal/igfs/hadoop/IgfsHadoop.java |  198 --
 .../IgfsHadoopCommunicationException.java       |   57 -
 .../igfs/hadoop/IgfsHadoopEndpoint.java         |  210 --
 .../internal/igfs/hadoop/IgfsHadoopEx.java      |   88 -
 .../igfs/hadoop/IgfsHadoopFSProperties.java     |   88 -
 .../internal/igfs/hadoop/IgfsHadoopFuture.java  |   94 -
 .../internal/igfs/hadoop/IgfsHadoopInProc.java  |  409 ----
 .../igfs/hadoop/IgfsHadoopInputStream.java      |  626 ------
 .../internal/igfs/hadoop/IgfsHadoopIo.java      |   76 -
 .../internal/igfs/hadoop/IgfsHadoopIpcIo.java   |  599 ------
 .../igfs/hadoop/IgfsHadoopIpcIoListener.java    |   36 -
 .../igfs/hadoop/IgfsHadoopJclLogger.java        |  112 -
 .../internal/igfs/hadoop/IgfsHadoopOutProc.java |  466 -----
 .../igfs/hadoop/IgfsHadoopOutputStream.java     |  201 --
 .../igfs/hadoop/IgfsHadoopProxyInputStream.java |  335 ---
 .../hadoop/IgfsHadoopProxyOutputStream.java     |  165 --
 .../internal/igfs/hadoop/IgfsHadoopReader.java  |  104 -
 .../igfs/hadoop/IgfsHadoopStreamDelegate.java   |   96 -
 .../hadoop/IgfsHadoopStreamEventListener.java   |   39 -
 .../internal/igfs/hadoop/IgfsHadoopUtils.java   |  131 --
 .../internal/igfs/hadoop/IgfsHadoopWrapper.java |  511 -----
 .../planner/HadoopDefaultMapReducePlanner.java  |    2 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java | 1967 ++++++++++++++++++
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   74 +
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   74 +
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |  304 +++
 .../igfs/HadoopIgfsDualAsyncSelfTest.java       |   32 +
 .../ignite/igfs/HadoopIgfsDualSyncSelfTest.java |   32 +
 .../IgfsHadoop20FileSystemAbstractSelfTest.java | 1967 ------------------
 ...doop20FileSystemLoopbackPrimarySelfTest.java |   74 -
 ...sHadoop20FileSystemShmemPrimarySelfTest.java |   74 -
 .../igfs/IgfsHadoopDualAbstractSelfTest.java    |  304 ---
 .../igfs/IgfsHadoopDualAsyncSelfTest.java       |   32 -
 .../ignite/igfs/IgfsHadoopDualSyncSelfTest.java |   32 -
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   18 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    6 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   12 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    6 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |    2 +-
 64 files changed, 7175 insertions(+), 7175 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopSecondaryFileSystem.java
index 5f06a65..9547e9f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopSecondaryFileSystem.java
@@ -157,7 +157,7 @@ public class IgniteHadoopSecondaryFileSystem implements Igfs, AutoCloseable {
 
     /** {@inheritDoc} */
     @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
-        IgfsHadoopFSProperties props0 = new IgfsHadoopFSProperties(props);
+        HadoopIgfsProperties props0 = new HadoopIgfsProperties(props);
 
         try {
             if (props0.userName() != null || props0.groupName() != null)
@@ -211,7 +211,7 @@ public class IgniteHadoopSecondaryFileSystem implements Igfs, AutoCloseable {
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
         try {
-            if (!fileSys.mkdirs(convert(path), new IgfsHadoopFSProperties(props).permission()))
+            if (!fileSys.mkdirs(convert(path), new HadoopIgfsProperties(props).permission()))
                 throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]");
         }
         catch (IOException e) {
@@ -272,7 +272,7 @@ public class IgniteHadoopSecondaryFileSystem implements Igfs, AutoCloseable {
 
     /** {@inheritDoc} */
     @Override public IgfsReader open(IgfsPath path, int bufSize) {
-        return new IgfsHadoopReader(fileSys, convert(path), bufSize);
+        return new HadoopIgfsReader(fileSys, convert(path), bufSize);
     }
 
     /** {@inheritDoc} */
@@ -288,8 +288,8 @@ public class IgniteHadoopSecondaryFileSystem implements Igfs, AutoCloseable {
     /** {@inheritDoc} */
     @Override public OutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
         long blockSize, @Nullable Map<String, String> props) {
-        IgfsHadoopFSProperties props0 =
-            new IgfsHadoopFSProperties(props != null ? props : Collections.<String, String>emptyMap());
+        HadoopIgfsProperties props0 =
+            new HadoopIgfsProperties(props != null ? props : Collections.<String, String>emptyMap());
 
         try {
             return fileSys.create(convert(path), props0.permission(), overwrite, bufSize, (short)replication, blockSize,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index f25b29f..9c95437 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -43,7 +43,7 @@ import static org.apache.ignite.IgniteFs.*;
 import static org.apache.ignite.configuration.IgfsConfiguration.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
+import static org.apache.ignite.internal.igfs.hadoop.HadoopIgfsUtils.*;
 
 /**
  * {@code IGFS} Hadoop 1.x file system driver over file system API. To use
@@ -95,7 +95,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
     private final AtomicBoolean closeGuard = new AtomicBoolean();
 
     /** Grid remote client. */
-    private IgfsHadoopWrapper rmtClient;
+    private HadoopIgfsWrapper rmtClient;
 
     /** User name for each thread. */
     private final ThreadLocal<String> userName = new ThreadLocal<String>(){
@@ -243,7 +243,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new IgfsHadoopWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -507,13 +507,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
                     clientLog.logOpen(logId, path, PROXY, bufSize, size);
 
-                    return new FSDataInputStream(new IgfsHadoopProxyInputStream(is, clientLog, logId));
+                    return new FSDataInputStream(new HadoopIgfsProxyInputStream(is, clientLog, logId));
                 }
                 else
                     return is;
             }
             else {
-                IgfsHadoopStreamDelegate stream = seqReadsBeforePrefetchOverride ?
+                HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
                     rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
 
                 long logId = -1;
@@ -528,7 +528,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                     LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
                         ", bufSize=" + bufSize + ']');
 
-                IgfsHadoopInputStream igfsIn = new IgfsHadoopInputStream(stream, stream.length(),
+                HadoopInputIgfsStream igfsIn = new HadoopInputIgfsStream(stream, stream.length(),
                     bufSize, LOG, clientLog, logId);
 
                 if (LOG.isDebugEnabled())
@@ -575,14 +575,14 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
                     clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
 
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(os, clientLog, logId));
+                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
                 }
                 else
                     return os;
             }
             else {
                 // Create stream and close it in the 'finally' section if any sequential operation failed.
-                IgfsHadoopStreamDelegate stream = rmtClient.create(path, overwrite, colocateFileWrites,
+                HadoopIgfsStreamDelegate stream = rmtClient.create(path, overwrite, colocateFileWrites,
                     replication, blockSize, F.asMap(PROP_PERMISSION, toString(perm),
                     PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites)));
 
@@ -599,7 +599,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                 if (LOG.isDebugEnabled())
                     LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
 
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG, clientLog,
+                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG, clientLog,
                     logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);
@@ -652,13 +652,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
                     clientLog.logAppend(logId, path, PROXY, bufSize); // Don't have stream ID.
 
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(os, clientLog, logId));
+                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
                 }
                 else
                     return os;
             }
             else {
-                IgfsHadoopStreamDelegate stream = rmtClient.append(path, false, null);
+                HadoopIgfsStreamDelegate stream = rmtClient.append(path, false, null);
 
                 assert stream != null;
 
@@ -673,7 +673,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                 if (LOG.isDebugEnabled())
                     LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
 
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG, clientLog,
+                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG, clientLog,
                     logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index 280af82..1c9165c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -43,7 +43,7 @@ import static org.apache.ignite.IgniteFs.*;
 import static org.apache.ignite.configuration.IgfsConfiguration.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
+import static org.apache.ignite.internal.igfs.hadoop.HadoopIgfsUtils.*;
 
 /**
  * {@code IGFS} Hadoop 2.x file system driver over file system API. To use
@@ -89,7 +89,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
     private final AtomicBoolean closeGuard = new AtomicBoolean();
 
     /** Grid remote client. */
-    private IgfsHadoopWrapper rmtClient;
+    private HadoopIgfsWrapper rmtClient;
 
     /** Working directory. */
     private IgfsPath workingDir;
@@ -137,7 +137,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
      * @throws IOException If initialization failed.
      */
     public IgniteHadoopFileSystem(URI name, Configuration cfg) throws URISyntaxException, IOException {
-        super(IgfsHadoopEndpoint.normalize(name), IGFS_SCHEME, false, -1);
+        super(HadoopIgfsEndpoint.normalize(name), IGFS_SCHEME, false, -1);
 
         uri = name;
 
@@ -239,7 +239,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new IgfsHadoopWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -446,13 +446,13 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
                     clientLog.logOpen(logId, path, PROXY, bufSize, size);
 
-                    return new FSDataInputStream(new IgfsHadoopProxyInputStream(is, clientLog, logId));
+                    return new FSDataInputStream(new HadoopIgfsProxyInputStream(is, clientLog, logId));
                 }
                 else
                     return is;
             }
             else {
-                IgfsHadoopStreamDelegate stream = seqReadsBeforePrefetchOverride ?
+                HadoopIgfsStreamDelegate stream = seqReadsBeforePrefetchOverride ?
                     rmtClient.open(path, seqReadsBeforePrefetch) : rmtClient.open(path);
 
                 long logId = -1;
@@ -467,7 +467,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
                     LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
                         ", bufSize=" + bufSize + ']');
 
-                IgfsHadoopInputStream igfsIn = new IgfsHadoopInputStream(stream, stream.length(),
+                HadoopInputIgfsStream igfsIn = new HadoopInputIgfsStream(stream, stream.length(),
                     bufSize, LOG, clientLog, logId);
 
                 if (LOG.isDebugEnabled())
@@ -524,7 +524,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
                     else
                         clientLog.logCreate(logId, path, PROXY, overwrite, bufSize, replication, blockSize);
 
-                    return new FSDataOutputStream(new IgfsHadoopProxyOutputStream(os, clientLog, logId));
+                    return new FSDataOutputStream(new HadoopIgfsProxyOutputStream(os, clientLog, logId));
                 }
                 else
                     return os;
@@ -534,7 +534,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
                     PROP_PREFER_LOCAL_WRITES, Boolean.toString(preferLocFileWrites));
 
                 // Create stream and close it in the 'finally' section if any sequential operation failed.
-                IgfsHadoopStreamDelegate stream;
+                HadoopIgfsStreamDelegate stream;
 
                 long logId = -1;
 
@@ -566,7 +566,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
                 assert stream != null;
 
-                IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(stream, LOG,
+                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG,
                     clientLog, logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfs.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfs.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfs.java
new file mode 100644
index 0000000..6ee593e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfs.java
@@ -0,0 +1,198 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Facade for communication with grid.
+ */
+public interface HadoopIgfs {
+    /**
+     * Perform handshake.
+     *
+     * @param logDir Log directory.
+     * @return Future with handshake result.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException, IOException;
+
+    /**
+     * Close connection.
+     *
+     * @param force Force flag.
+     */
+    public void close(boolean force);
+
+    /**
+     * Command to retrieve file info for some IGFS path.
+     *
+     * @param path Path to get file info for.
+     * @return Future for info operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsFile info(IgfsPath path) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to update file properties.
+     *
+     * @param path IGFS path to update properties.
+     * @param props Properties to update.
+     * @return Future for update operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException, IOException;
+
+    /**
+     * Sets last access time and last modification time for a file.
+     *
+     * @param path Path to update times.
+     * @param accessTime Last access time to set.
+     * @param modificationTime Last modification time to set.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException,
+        IOException;
+
+    /**
+     * Command to rename given path.
+     *
+     * @param src Source path.
+     * @param dest Destination path.
+     * @return Future for rename operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to delete given path.
+     *
+     * @param path Path to delete.
+     * @param recursive {@code True} if deletion is recursive.
+     * @return Future for delete operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to get affinity for given path, offset and length.
+     *
+     * @param path Path to get affinity for.
+     * @param start Start position (offset).
+     * @param len Data length.
+     * @return Future for affinity command.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) throws IgniteCheckedException,
+        IOException;
+
+    /**
+     * Gets path summary.
+     *
+     * @param path Path to get summary for.
+     * @return Future that will be completed when summary is received.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to create directories.
+     *
+     * @param path Path to create.
+     * @return Future for mkdirs operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to get list of files in directory.
+     *
+     * @param path Path to list.
+     * @return Future for listFiles operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to get directory listing.
+     *
+     * @param path Path to list.
+     * @return Future for listPaths operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException, IOException;
+
+    /**
+     * Performs status request.
+     *
+     * @return Status response.
+     * @throws IgniteCheckedException If failed.
+     */
+    public IgfsStatus fsStatus() throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to open file for reading.
+     *
+     * @param path File path to open.
+     * @return Future for open operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException, IOException;
+
+    /**
+     * Command to open file for reading.
+     *
+     * @param path File path to open.
+     * @return Future for open operation.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopIgfsStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch) throws IgniteCheckedException,
+        IOException;
+
+    /**
+     * Command to create file and open it for output.
+     *
+     * @param path Path to file.
+     * @param overwrite If {@code true} then old file contents will be lost.
+     * @param colocate If {@code true} and called on data node, file will be written on that node.
+     * @param replication Replication factor.
+     * @param props File properties for creation.
+     * @return Stream descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
+        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException, IOException;
+
+    /**
+     * Open file for output appending data to the end of a file.
+     *
+     * @param path Path to file.
+     * @param create If {@code true}, file will be created if does not exist.
+     * @param props File properties.
+     * @return Stream descriptor.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
+        @Nullable Map<String, String> props) throws IgniteCheckedException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsCommunicationException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsCommunicationException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsCommunicationException.java
new file mode 100644
index 0000000..ecaa61f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsCommunicationException.java
@@ -0,0 +1,57 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.*;
+
+/**
+ * Communication exception indicating a problem between file system and IGFS instance.
+ */
+public class HadoopIgfsCommunicationException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates new exception with given throwable as a nested cause and
+     * source of error message.
+     *
+     * @param cause Non-null throwable cause.
+     */
+    public HadoopIgfsCommunicationException(Exception cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested cause exception.
+     *
+     * @param msg Error message.
+     */
+    public HadoopIgfsCommunicationException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates a new exception with given error message and optional nested cause exception.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public HadoopIgfsCommunicationException(String msg, Exception cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEndpoint.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEndpoint.java
new file mode 100644
index 0000000..dc8fcb8
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEndpoint.java
@@ -0,0 +1,210 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+
+import static org.apache.ignite.configuration.IgfsConfiguration.*;
+
+/**
+ * IGFS endpoint abstraction.
+ */
+public class HadoopIgfsEndpoint {
+    /** Localhost. */
+    public static final String LOCALHOST = "127.0.0.1";
+
+    /** IGFS name. */
+    private final String igfsName;
+
+    /** Grid name. */
+    private final String gridName;
+
+    /** Host. */
+    private final String host;
+
+    /** Port. */
+    private final int port;
+
+    /**
+     * Normalize IGFS URI.
+     *
+     * @param uri URI.
+     * @return Normalized URI.
+     * @throws IOException If failed.
+     */
+    public static URI normalize(URI uri) throws IOException {
+        try {
+            if (!F.eq(IgniteFs.IGFS_SCHEME, uri.getScheme()))
+                throw new IOException("Failed to normalize UIR because it has non IGFS scheme: " + uri);
+
+            HadoopIgfsEndpoint endpoint = new HadoopIgfsEndpoint(uri.getAuthority());
+
+            StringBuilder sb = new StringBuilder();
+
+            if (endpoint.igfs() != null)
+                sb.append(endpoint.igfs());
+
+            if (endpoint.grid() != null)
+                sb.append(":").append(endpoint.grid());
+
+            return new URI(uri.getScheme(), sb.length() != 0 ? sb.toString() : null, endpoint.host(), endpoint.port(),
+                uri.getPath(), uri.getQuery(), uri.getFragment());
+        }
+        catch (URISyntaxException | IgniteCheckedException e) {
+            throw new IOException("Failed to normalize URI: " + uri, e);
+        }
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param connStr Connection string.
+     * @throws IgniteCheckedException If failed to parse connection string.
+     */
+    public HadoopIgfsEndpoint(@Nullable String connStr) throws IgniteCheckedException {
+        if (connStr == null)
+            connStr = "";
+
+        String[] tokens = connStr.split("@", -1);
+
+        IgniteBiTuple<String, Integer> hostPort;
+
+        if (tokens.length == 1) {
+            igfsName = null;
+            gridName = null;
+
+            hostPort = hostPort(connStr, connStr);
+        }
+        else if (tokens.length == 2) {
+            String authStr = tokens[0];
+
+            if (authStr.isEmpty()) {
+                gridName = null;
+                igfsName = null;
+            }
+            else {
+                String[] authTokens = authStr.split(":", -1);
+
+                igfsName = F.isEmpty(authTokens[0]) ? null : authTokens[0];
+
+                if (authTokens.length == 1)
+                    gridName = null;
+                else if (authTokens.length == 2)
+                    gridName = F.isEmpty(authTokens[1]) ? null : authTokens[1];
+                else
+                    throw new IgniteCheckedException("Invalid connection string format: " + connStr);
+            }
+
+            hostPort = hostPort(connStr, tokens[1]);
+        }
+        else
+            throw new IgniteCheckedException("Invalid connection string format: " + connStr);
+
+        host = hostPort.get1();
+
+        assert hostPort.get2() != null;
+
+        port = hostPort.get2();
+    }
+
+    /**
+     * Parse host and port.
+     *
+     * @param connStr Full connection string.
+     * @param hostPortStr Host/port connection string part.
+     * @return Tuple with host and port.
+     * @throws IgniteCheckedException If failed to parse connection string.
+     */
+    private IgniteBiTuple<String, Integer> hostPort(String connStr, String hostPortStr) throws IgniteCheckedException {
+        String[] tokens = hostPortStr.split(":", -1);
+
+        String host = tokens[0];
+
+        if (F.isEmpty(host))
+            host = LOCALHOST;
+
+        int port;
+
+        if (tokens.length == 1)
+            port = DFLT_IPC_PORT;
+        else if (tokens.length == 2) {
+            String portStr = tokens[1];
+
+            try {
+                port = Integer.valueOf(portStr);
+
+                if (port < 0 || port > 65535)
+                    throw new IgniteCheckedException("Invalid port number: " + connStr);
+            }
+            catch (NumberFormatException e) {
+                throw new IgniteCheckedException("Invalid port number: " + connStr);
+            }
+        }
+        else
+            throw new IgniteCheckedException("Invalid connection string format: " + connStr);
+
+        return F.t(host, port);
+    }
+
+    /**
+     * @return IGFS name.
+     */
+    @Nullable public String igfs() {
+        return igfsName;
+    }
+
+    /**
+     * @return Grid name.
+     */
+    @Nullable public String grid() {
+        return gridName;
+    }
+
+    /**
+     * @return Host.
+     */
+    public String host() {
+        return host;
+    }
+
+    /**
+     * @return Host.
+     */
+    public boolean isLocal() {
+        return F.eq(LOCALHOST, host);
+    }
+
+    /**
+     * @return Port.
+     */
+    public int port() {
+        return port;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopIgfsEndpoint.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEx.java
new file mode 100644
index 0000000..5321fa3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsEx.java
@@ -0,0 +1,88 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.internal.util.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Extended IGFS server interface.
+ */
+public interface HadoopIgfsEx extends HadoopIgfs {
+    /**
+     * Adds event listener that will be invoked when connection with server is lost or remote error has occurred.
+     * If connection is closed already, callback will be invoked synchronously inside this method.
+     *
+     * @param delegate Stream delegate.
+     * @param lsnr Event listener.
+     */
+    public void addEventListener(HadoopIgfsStreamDelegate delegate, HadoopIgfsStreamEventListener lsnr);
+
+    /**
+     * Removes event listener that will be invoked when connection with server is lost or remote error has occurred.
+     *
+     * @param delegate Stream delegate.
+     */
+    public void removeEventListener(HadoopIgfsStreamDelegate delegate);
+
+    /**
+     * Asynchronously reads specified amount of bytes from opened input stream.
+     *
+     * @param delegate Stream delegate.
+     * @param pos Position to read from.
+     * @param len Data length to read.
+     * @param outBuf Optional output buffer. If buffer length is less then {@code len}, all remaining
+     *     bytes will be read into new allocated buffer of length {len - outBuf.length} and this buffer will
+     *     be the result of read future.
+     * @param outOff Output offset.
+     * @param outLen Output length.
+     * @return Read data.
+     */
+    public GridPlainFuture<byte[]> readData(HadoopIgfsStreamDelegate delegate, long pos, int len,
+        @Nullable final byte[] outBuf, final int outOff, final int outLen);
+
+    /**
+     * Writes data to the stream with given streamId. This method does not return any future since
+     * no response to write request is sent.
+     *
+     * @param delegate Stream delegate.
+     * @param data Data to write.
+     * @param off Offset.
+     * @param len Length.
+     * @throws IOException If failed.
+     */
+    public void writeData(HadoopIgfsStreamDelegate delegate, byte[] data, int off, int len) throws IOException;
+
+    /**
+     * Close server stream.
+     *
+     * @param delegate Stream delegate.
+     * @throws IOException If failed.
+     */
+    public void closeStream(HadoopIgfsStreamDelegate delegate) throws IOException;
+
+    /**
+     * Flush output stream.
+     *
+     * @param delegate Stream delegate.
+     * @throws IOException If failed.
+     */
+    public void flush(HadoopIgfsStreamDelegate delegate) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsFuture.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsFuture.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsFuture.java
new file mode 100644
index 0000000..9ae0161
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsFuture.java
@@ -0,0 +1,94 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.internal.util.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * IGFS client future that holds response parse closure.
+ */
+public class HadoopIgfsFuture<T> extends GridPlainFutureAdapter<T> {
+    /** Output buffer. */
+    private byte[] outBuf;
+
+    /** Output offset. */
+    private int outOff;
+
+    /** Output length. */
+    private int outLen;
+
+    /** Read future flag. */
+    private boolean read;
+
+    /**
+     * @return Output buffer.
+     */
+    public byte[] outputBuffer() {
+        return outBuf;
+    }
+
+    /**
+     * @param outBuf Output buffer.
+     */
+    public void outputBuffer(@Nullable byte[] outBuf) {
+        this.outBuf = outBuf;
+    }
+
+    /**
+     * @return Offset in output buffer to write from.
+     */
+    public int outputOffset() {
+        return outOff;
+    }
+
+    /**
+     * @param outOff Offset in output buffer to write from.
+     */
+    public void outputOffset(int outOff) {
+        this.outOff = outOff;
+    }
+
+    /**
+     * @return Length to write to output buffer.
+     */
+    public int outputLength() {
+        return outLen;
+    }
+
+    /**
+     * @param outLen Length to write to output buffer.
+     */
+    public void outputLength(int outLen) {
+        this.outLen = outLen;
+    }
+
+    /**
+     * @param read {@code True} if this is a read future.
+     */
+    public void read(boolean read) {
+        this.read = read;
+    }
+
+    /**
+     * @return {@code True} if this is a read future.
+     */
+    public boolean read() {
+        return read;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInProc.java
new file mode 100644
index 0000000..b0da50d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInProc.java
@@ -0,0 +1,409 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Communication with grid in the same process.
+ */
+public class HadoopIgfsInProc implements HadoopIgfsEx {
+    /** Target IGFS. */
+    private final IgfsEx igfs;
+
+    /** Buffer size. */
+    private final int bufSize;
+
+    /** Event listeners. */
+    private final Map<HadoopIgfsStreamDelegate, HadoopIgfsStreamEventListener> lsnrs =
+        new ConcurrentHashMap<>();
+
+    /** Logger. */
+    private final Log log;
+
+    /**
+     * Constructor.
+     *
+     * @param igfs Target IGFS.
+     * @param log Log.
+     */
+    public HadoopIgfsInProc(IgfsEx igfs, Log log) {
+        this.igfs = igfs;
+        this.log = log;
+
+        bufSize = igfs.configuration().getBlockSize() * 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsHandshakeResponse handshake(String logDir) {
+        igfs.clientLogDirectory(logDir);
+
+        return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
+            igfs.globalSampling());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(boolean force) {
+        // Perform cleanup.
+        for (HadoopIgfsStreamEventListener lsnr : lsnrs.values()) {
+            try {
+                lsnr.onClose();
+            }
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to notify stream event listener", e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
+        try {
+            return igfs.info(path);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to get file info because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        try {
+            return igfs.update(path, props);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to update file because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+        try {
+            igfs.setTimes(path, accessTime, modificationTime);
+
+            return true;
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to set path times because Grid is stopping: " +
+                path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
+        try {
+            igfs.rename(src, dest);
+
+            return true;
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to rename path because Grid is stopping: " + src);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
+        try {
+            return igfs.delete(path, recursive);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to delete path because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
+        try {
+            return igfs.globalSpace();
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to get file system status because Grid is " +
+                "stopping.");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
+        try {
+            return igfs.listPaths(path);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to list paths because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
+        try {
+            return igfs.listFiles(path);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to list files because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        try {
+            igfs.mkdirs(path, props);
+
+            return true;
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to create directory because Grid is stopping: " +
+                path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
+        try {
+            return igfs.summary(path);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to get content summary because Grid is stopping: " +
+                path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
+        throws IgniteCheckedException {
+        try {
+            return igfs.affinity(path, start, len);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to get affinity because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
+        try {
+            IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
+
+            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to open file because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch)
+        throws IgniteCheckedException {
+        try {
+            IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
+
+            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to open file because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
+        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+        try {
+            IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
+                colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
+
+            return new HadoopIgfsStreamDelegate(this, stream);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to create file because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
+        @Nullable Map<String, String> props) throws IgniteCheckedException {
+        try {
+            IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
+
+            return new HadoopIgfsStreamDelegate(this, stream);
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (IllegalStateException e) {
+            throw new HadoopIgfsCommunicationException("Failed to append file because Grid is stopping: " + path);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPlainFuture<byte[]> readData(HadoopIgfsStreamDelegate delegate, long pos, int len,
+        @Nullable byte[] outBuf, int outOff, int outLen) {
+        IgfsInputStreamAdapter stream = delegate.target();
+
+        try {
+            byte[] res = null;
+
+            if (outBuf != null) {
+                int outTailLen = outBuf.length - outOff;
+
+                if (len <= outTailLen)
+                    stream.readFully(pos, outBuf, outOff, len);
+                else {
+                    stream.readFully(pos, outBuf, outOff, outTailLen);
+
+                    int remainderLen = len - outTailLen;
+
+                    res = new byte[remainderLen];
+
+                    stream.readFully(pos, res, 0, remainderLen);
+                }
+            } else {
+                res = new byte[len];
+
+                stream.readFully(pos, res, 0, len);
+            }
+
+            return new GridPlainFutureAdapter<>(res);
+        }
+        catch (IllegalStateException | IOException e) {
+            HadoopIgfsStreamEventListener lsnr = lsnrs.get(delegate);
+
+            if (lsnr != null)
+                lsnr.onError(e.getMessage());
+
+            return new GridPlainFutureAdapter<>(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeData(HadoopIgfsStreamDelegate delegate, byte[] data, int off, int len)
+        throws IOException {
+        try {
+            IgfsOutputStream stream = delegate.target();
+
+            stream.write(data, off, len);
+        }
+        catch (IllegalStateException | IOException e) {
+            HadoopIgfsStreamEventListener lsnr = lsnrs.get(delegate);
+
+            if (lsnr != null)
+                lsnr.onError(e.getMessage());
+
+            if (e instanceof IllegalStateException)
+                throw new IOException("Failed to write data to IGFS stream because Grid is stopping.", e);
+            else
+                throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush(HadoopIgfsStreamDelegate delegate) throws IOException {
+        try {
+            IgfsOutputStream stream = delegate.target();
+
+            stream.flush();
+        }
+        catch (IllegalStateException | IOException e) {
+            HadoopIgfsStreamEventListener lsnr = lsnrs.get(delegate);
+
+            if (lsnr != null)
+                lsnr.onError(e.getMessage());
+
+            if (e instanceof IllegalStateException)
+                throw new IOException("Failed to flush data to IGFS stream because Grid is stopping.", e);
+            else
+                throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeStream(HadoopIgfsStreamDelegate desc) throws IOException {
+        Closeable closeable = desc.target();
+
+        try {
+            closeable.close();
+        }
+        catch (IllegalStateException e) {
+            throw new IOException("Failed to close IGFS stream because Grid is stopping.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addEventListener(HadoopIgfsStreamDelegate delegate,
+        HadoopIgfsStreamEventListener lsnr) {
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.put(delegate, lsnr);
+
+        assert lsnr0 == null || lsnr0 == lsnr;
+
+        if (log.isDebugEnabled())
+            log.debug("Added stream event listener [delegate=" + delegate + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeEventListener(HadoopIgfsStreamDelegate delegate) {
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.remove(delegate);
+
+        if (lsnr0 != null && log.isDebugEnabled())
+            log.debug("Removed stream event listener [delegate=" + delegate + ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIo.java
new file mode 100644
index 0000000..775e7d0
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIo.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.igfs.hadoop;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * IO abstraction layer for IGFS client. Two kind of messages are expected to be sent: requests with response
+ * and request without response.
+ */
+public interface HadoopIgfsIo {
+    /**
+     * Sends given IGFS client message and asynchronously awaits for response.
+     *
+     * @param msg Message to send.
+     * @return Future that will be completed.
+     * @throws IgniteCheckedException If a message cannot be sent (connection is broken or client was closed).
+     */
+    public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException;
+
+    /**
+     * Sends given IGFS client message and asynchronously awaits for response. When IO detects response
+     * beginning for given message it stops reading data and passes input stream to closure which can read
+     * response in a specific way.
+     *
+     * @param msg Message to send.
+     * @param outBuf Output buffer. If {@code null}, the output buffer is not used.
+     * @param outOff Output buffer offset.
+     * @param outLen Output buffer length.
+     * @return Future that will be completed when response is returned from closure.
+     * @throws IgniteCheckedException If a message cannot be sent (connection is broken or client was closed).
+     */
+    public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff, int outLen)
+        throws IgniteCheckedException;
+
+    /**
+     * Sends given message and does not wait for response.
+     *
+     * @param msg Message to send.
+     * @throws IgniteCheckedException If send failed.
+     */
+    public void sendPlain(IgfsMessage msg) throws IgniteCheckedException;
+
+    /**
+     * Adds event listener that will be invoked when connection with server is lost or remote error has occurred.
+     * If connection is closed already, callback will be invoked synchronously inside this method.
+     *
+     * @param lsnr Event listener.
+     */
+    public void addEventListener(HadoopIgfsIpcIoListener lsnr);
+
+    /**
+     * Removes event listener that will be invoked when connection with server is lost or remote error has occurred.
+     *
+     * @param lsnr Event listener.
+     */
+    public void removeEventListener(HadoopIgfsIpcIoListener lsnr);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
new file mode 100644
index 0000000..10d764e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
@@ -0,0 +1,36 @@
+/*
+ * 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.igfs.hadoop;
+
+/**
+ * Listens to the events of {@link HadoopIpcIgfsIo}.
+ */
+public interface HadoopIgfsIpcIoListener {
+    /**
+     * Callback invoked when the IO is being closed.
+     */
+    public void onClose();
+
+    /**
+     * Callback invoked when remote error occurs.
+     *
+     * @param streamId Stream ID.
+     * @param errMsg Error message.
+     */
+    public void onError(long streamId, String errMsg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsJclLogger.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsJclLogger.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsJclLogger.java
new file mode 100644
index 0000000..1ba6e64
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsJclLogger.java
@@ -0,0 +1,112 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * JCL logger wrapper for Hadoop.
+ */
+public class HadoopIgfsJclLogger implements IgniteLogger {
+    /** JCL implementation proxy. */
+    private Log impl;
+
+    /**
+     * Constructor.
+     *
+     * @param impl JCL implementation to use.
+     */
+    HadoopIgfsJclLogger(Log impl) {
+        assert impl != null;
+
+        this.impl = impl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteLogger getLogger(Object ctgr) {
+        return new HadoopIgfsJclLogger(LogFactory.getLog(
+            ctgr instanceof Class ? ((Class)ctgr).getName() : String.valueOf(ctgr)));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void trace(String msg) {
+        impl.trace(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void debug(String msg) {
+        impl.debug(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void info(String msg) {
+        impl.info(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warning(String msg) {
+        impl.warn(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void warning(String msg, @Nullable Throwable e) {
+        impl.warn(msg, e);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(String msg) {
+        impl.error(msg);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isQuiet() {
+        return !isInfoEnabled() && !isDebugEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void error(String msg, @Nullable Throwable e) {
+        impl.error(msg, e);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTraceEnabled() {
+        return impl.isTraceEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isDebugEnabled() {
+        return impl.isDebugEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isInfoEnabled() {
+        return impl.isInfoEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String fileName() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return "IgfsHadoopJclLogger [impl=" + impl + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProperties.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProperties.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProperties.java
new file mode 100644
index 0000000..20c1d5d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProperties.java
@@ -0,0 +1,88 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.hadoop.fs.permission.*;
+import org.apache.ignite.*;
+
+import java.util.*;
+
+import static org.apache.ignite.IgniteFs.*;
+
+/**
+ * Hadoop file system properties.
+ */
+public class HadoopIgfsProperties {
+    /** Username. */
+    private String usrName;
+
+    /** Group name. */
+    private String grpName;
+
+    /** Permissions. */
+    private FsPermission perm;
+
+    /**
+     * Constructor.
+     *
+     * @param props Properties.
+     * @throws IgniteException In case of error.
+     */
+    public HadoopIgfsProperties(Map<String, String> props) throws IgniteException {
+        usrName = props.get(PROP_USER_NAME);
+        grpName = props.get(PROP_GROUP_NAME);
+
+        String permStr = props.get(PROP_PERMISSION);
+
+        if (permStr != null) {
+            try {
+                perm = new FsPermission((short)Integer.parseInt(permStr, 8));
+            }
+            catch (NumberFormatException ignore) {
+                throw new IgniteException("Permissions cannot be parsed: " + permStr);
+            }
+        }
+    }
+
+    /**
+     * Get user name.
+     *
+     * @return User name.
+     */
+    public String userName() {
+        return usrName;
+    }
+
+    /**
+     * Get group name.
+     *
+     * @return Group name.
+     */
+    public String groupName() {
+        return grpName;
+    }
+
+    /**
+     * Get permission.
+     *
+     * @return Permission.
+     */
+    public FsPermission permission() {
+        return perm;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyInputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyInputStream.java
new file mode 100644
index 0000000..cfc6949
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyInputStream.java
@@ -0,0 +1,335 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.internal.igfs.common.*;
+
+import java.io.*;
+
+/**
+ * Secondary Hadoop file system input stream wrapper.
+ */
+public class HadoopIgfsProxyInputStream extends InputStream implements Seekable, PositionedReadable {
+    /** Actual input stream to the secondary file system. */
+    private final FSDataInputStream is;
+
+    /** Client logger. */
+    private final IgfsLogger clientLog;
+
+    /** Log stream ID. */
+    private final long logStreamId;
+
+    /** Read time. */
+    private long readTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of read bytes. */
+    private long total;
+
+    /** Closed flag. */
+    private boolean closed;
+
+    /**
+     * Constructor.
+     *
+     * @param is Actual input stream to the secondary file system.
+     * @param clientLog Client log.
+     */
+    public HadoopIgfsProxyInputStream(FSDataInputStream is, IgfsLogger clientLog, long logStreamId) {
+        assert is != null;
+        assert clientLog != null;
+
+        this.is = is;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        lastTs = System.nanoTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(byte[] b) throws IOException {
+        readStart();
+
+        int res;
+
+        try {
+            res = is.read(b);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (res != -1)
+            total += res;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(byte[] b, int off, int len) throws IOException {
+        readStart();
+
+        int res;
+
+        try {
+            res = super.read(b, off, len);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (res != -1)
+            total += res;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long skip(long n) throws IOException {
+        readStart();
+
+        long res;
+
+        try {
+            res =  is.skip(n);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSkip(logStreamId, res);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int available() throws IOException {
+        readStart();
+
+        try {
+            return is.available();
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void close() throws IOException {
+        if (!closed) {
+            closed = true;
+
+            readStart();
+
+            try {
+                is.close();
+            }
+            finally {
+                readEnd();
+            }
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void mark(int readLimit) {
+        readStart();
+
+        try {
+            is.mark(readLimit);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (clientLog.isLogEnabled())
+            clientLog.logMark(logStreamId, readLimit);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void reset() throws IOException {
+        readStart();
+
+        try {
+            is.reset();
+        }
+        finally {
+            readEnd();
+        }
+
+        if (clientLog.isLogEnabled())
+            clientLog.logReset(logStreamId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean markSupported() {
+        readStart();
+
+        try {
+            return is.markSupported();
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read() throws IOException {
+        readStart();
+
+        int res;
+
+        try {
+            res = is.read();
+        }
+        finally {
+            readEnd();
+        }
+
+        if (res != -1)
+            total++;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(long pos, byte[] buf, int off, int len) throws IOException {
+        readStart();
+
+        int res;
+
+        try {
+            res = is.read(pos, buf, off, len);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (res != -1)
+            total += res;
+
+        if (clientLog.isLogEnabled())
+            clientLog.logRandomRead(logStreamId, pos, res);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void readFully(long pos, byte[] buf, int off, int len) throws IOException {
+        readStart();
+
+        try {
+            is.readFully(pos, buf, off, len);
+        }
+        finally {
+            readEnd();
+        }
+
+        total += len;
+
+        if (clientLog.isLogEnabled())
+            clientLog.logRandomRead(logStreamId, pos, len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void readFully(long pos, byte[] buf) throws IOException {
+        readStart();
+
+        try {
+            is.readFully(pos, buf);
+        }
+        finally {
+            readEnd();
+        }
+
+        total += buf.length;
+
+        if (clientLog.isLogEnabled())
+            clientLog.logRandomRead(logStreamId, pos, buf.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void seek(long pos) throws IOException {
+        readStart();
+
+        try {
+            is.seek(pos);
+        }
+        finally {
+            readEnd();
+        }
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSeek(logStreamId, pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long getPos() throws IOException {
+        readStart();
+
+        try {
+            return is.getPos();
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+        readStart();
+
+        try {
+            return is.seekToNewSource(targetPos);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /**
+     * Read start.
+     */
+    private void readStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void readEnd() {
+        long now = System.nanoTime();
+
+        readTime += now - lastTs;
+
+        lastTs = now;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyOutputStream.java
new file mode 100644
index 0000000..a7266c4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsProxyOutputStream.java
@@ -0,0 +1,165 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.internal.igfs.common.*;
+
+import java.io.*;
+
+/**
+ * Secondary Hadoop file system output stream wrapper.
+ */
+public class HadoopIgfsProxyOutputStream extends OutputStream {
+    /** Actual output stream. */
+    private FSDataOutputStream os;
+
+    /** Client logger. */
+    private final IgfsLogger clientLog;
+
+    /** Log stream ID. */
+    private final long logStreamId;
+
+    /** Read time. */
+    private long writeTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of written bytes. */
+    private long total;
+
+    /** Closed flag. */
+    private boolean closed;
+
+    /**
+     * Constructor.
+     *
+     * @param os Actual output stream.
+     * @param clientLog Client logger.
+     * @param logStreamId Log stream ID.
+     */
+    public HadoopIgfsProxyOutputStream(FSDataOutputStream os, IgfsLogger clientLog, long logStreamId) {
+        assert os != null;
+        assert clientLog != null;
+
+        this.os = os;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        lastTs = System.nanoTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void write(int b) throws IOException {
+        writeStart();
+
+        try {
+            os.write(b);
+        }
+        finally {
+            writeEnd();
+        }
+
+        total++;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void write(byte[] b) throws IOException {
+        writeStart();
+
+        try {
+            os.write(b);
+        }
+        finally {
+            writeEnd();
+        }
+
+        total += b.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void write(byte[] b, int off, int len) throws IOException {
+        writeStart();
+
+        try {
+            os.write(b, off, len);
+        }
+        finally {
+            writeEnd();
+        }
+
+        total += len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void flush() throws IOException {
+        writeStart();
+
+        try {
+            os.flush();
+        }
+        finally {
+            writeEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void close() throws IOException {
+        if (!closed) {
+            closed = true;
+
+            writeStart();
+
+            try {
+                os.close();
+            }
+            finally {
+                writeEnd();
+            }
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
+        }
+    }
+
+    /**
+     * Read start.
+     */
+    private void writeStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void writeEnd() {
+        long now = System.nanoTime();
+
+        writeTime += now - lastTs;
+
+        lastTs = now;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsReader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsReader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsReader.java
new file mode 100644
index 0000000..17755dc
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsReader.java
@@ -0,0 +1,104 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Secondary file system input stream wrapper which actually opens input stream only in case it is explicitly
+ * requested.
+ * <p>
+ * The class is expected to be used only from synchronized context and therefore is not tread-safe.
+ */
+public class HadoopIgfsReader implements IgfsReader {
+    /** Secondary file system. */
+    private final FileSystem fs;
+
+    /** Path to the file to open. */
+    private final Path path;
+
+    /** Buffer size. */
+    private final int bufSize;
+
+    /** Actual input stream. */
+    private FSDataInputStream in;
+
+    /** Cached error occurred during output stream open. */
+    private IOException err;
+
+    /** Flag indicating that the stream was already opened. */
+    private boolean opened;
+
+    /**
+     * Constructor.
+     *
+     * @param fs Secondary file system.
+     * @param path Path to the file to open.
+     * @param bufSize Buffer size.
+     */
+    public HadoopIgfsReader(FileSystem fs, Path path, int bufSize) {
+        assert fs != null;
+        assert path != null;
+
+        this.fs = fs;
+        this.path = path;
+        this.bufSize = bufSize;
+    }
+
+    /** Get input stream. */
+    private PositionedReadable in() throws IOException {
+        if (opened) {
+            if (err != null)
+                throw err;
+        }
+        else {
+            opened = true;
+
+            try {
+                in = fs.open(path, bufSize);
+
+                if (in == null)
+                    throw new IOException("Failed to open input stream (file system returned null): " + path);
+            }
+            catch (IOException e) {
+                err = e;
+
+                throw err;
+            }
+        }
+
+        return in;
+    }
+
+    /**
+     * Close wrapped input stream in case it was previously opened.
+     */
+    @Override public void close() {
+        U.closeQuiet(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(long pos, byte[] buf, int off, int len) throws IOException {
+        return in().read(pos, buf, off, len);
+    }
+}


[03/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
new file mode 100644
index 0000000..91a2d6f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -0,0 +1,1625 @@
+/*
+ * 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.jobtracker;
+
+import org.apache.ignite.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.eventstorage.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.event.*;
+import javax.cache.expiry.*;
+import javax.cache.processor.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
+import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
+
+/**
+ * Hadoop job tracker.
+ */
+public class HadoopJobTracker extends HadoopComponent {
+    /** */
+    private final GridMutex mux = new GridMutex();
+
+    /** */
+    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaPrj;
+
+    /** Projection with expiry policy for finished job updates. */
+    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaPrj;
+
+    /** Map-reduce execution planner. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private GridHadoopMapReducePlanner mrPlanner;
+
+    /** All the known jobs. */
+    private final ConcurrentMap<GridHadoopJobId, GridFutureAdapterEx<GridHadoopJob>> jobs = new ConcurrentHashMap8<>();
+
+    /** Locally active jobs. */
+    private final ConcurrentMap<GridHadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
+
+    /** Locally requested finish futures. */
+    private final ConcurrentMap<GridHadoopJobId, GridFutureAdapter<GridHadoopJobId>> activeFinishFuts =
+        new ConcurrentHashMap8<>();
+
+    /** Event processing service. */
+    private ExecutorService evtProcSvc;
+
+    /** Component busy lock. */
+    private GridSpinReadWriteLock busyLock;
+
+    /** Closure to check result of async transform of system cache. */
+    private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
+        @Override public void apply(IgniteInternalFuture<?> gridFut) {
+            try {
+                gridFut.get();
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to transform system cache.", e);
+            }
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public void start(HadoopContext ctx) throws IgniteCheckedException {
+        super.start(ctx);
+
+        busyLock = new GridSpinReadWriteLock();
+
+        evtProcSvc = Executors.newFixedThreadPool(1);
+    }
+
+    /**
+     * @return Job meta projection.
+     */
+    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaCache() {
+        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = jobMetaPrj;
+
+        if (prj == null) {
+            synchronized (mux) {
+                if ((prj = jobMetaPrj) == null) {
+                    CacheProjection<Object, Object> sysCache = ctx.kernalContext().cache()
+                        .cache(CU.SYS_CACHE_HADOOP_MR);
+
+                    assert sysCache != null;
+
+                    mrPlanner = ctx.planner();
+
+                    try {
+                        ctx.kernalContext().resource().injectGeneric(mrPlanner);
+                    }
+                    catch (IgniteCheckedException e) { // Must not happen.
+                        U.error(log, "Failed to inject resources.", e);
+
+                        throw new IllegalStateException(e);
+                    }
+
+                    jobMetaPrj = prj = (GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata>)
+                        sysCache.projection(GridHadoopJobId.class, GridHadoopJobMetadata.class);
+
+                    if (ctx.configuration().getFinishedJobInfoTtl() > 0) {
+                        ExpiryPolicy finishedJobPlc = new ModifiedExpiryPolicy(
+                            new Duration(MILLISECONDS, ctx.configuration().getFinishedJobInfoTtl()));
+
+                        finishedJobMetaPrj = prj.withExpiryPolicy(finishedJobPlc);
+                    }
+                    else
+                        finishedJobMetaPrj = jobMetaPrj;
+                }
+            }
+        }
+
+        return prj;
+    }
+
+    /**
+     * @return Projection with expiry policy for finished job updates.
+     */
+    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaCache() {
+        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = finishedJobMetaPrj;
+
+        if (prj == null) {
+            jobMetaCache();
+
+            prj = finishedJobMetaPrj;
+
+            assert prj != null;
+        }
+
+        return prj;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        super.onKernalStart();
+
+        jobMetaCache().context().continuousQueries().executeInternalQuery(
+            new CacheEntryUpdatedListener<GridHadoopJobId, GridHadoopJobMetadata>() {
+                @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends GridHadoopJobId,
+                    ? extends GridHadoopJobMetadata>> evts) {
+                    if (!busyLock.tryReadLock())
+                        return;
+
+                    try {
+                        // Must process query callback in a separate thread to avoid deadlocks.
+                        evtProcSvc.submit(new EventHandler() {
+                            @Override protected void body() throws IgniteCheckedException {
+                                processJobMetadataUpdates(evts);
+                            }
+                        });
+                    }
+                    finally {
+                        busyLock.readUnlock();
+                    }
+                }
+            },
+            null,
+            true,
+            true
+        );
+
+        ctx.kernalContext().event().addLocalEventListener(new GridLocalEventListener() {
+            @Override public void onEvent(final Event evt) {
+                if (!busyLock.tryReadLock())
+                    return;
+
+                try {
+                    // Must process discovery callback in a separate thread to avoid deadlock.
+                    evtProcSvc.submit(new EventHandler() {
+                        @Override protected void body() {
+                            processNodeLeft((DiscoveryEvent)evt);
+                        }
+                    });
+                }
+                finally {
+                    busyLock.readUnlock();
+                }
+            }
+        }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        super.onKernalStop(cancel);
+
+        busyLock.writeLock();
+
+        evtProcSvc.shutdown();
+
+        // Fail all pending futures.
+        for (GridFutureAdapter<GridHadoopJobId> fut : activeFinishFuts.values())
+            fut.onDone(new IgniteCheckedException("Failed to execute Hadoop map-reduce job (grid is stopping)."));
+    }
+
+    /**
+     * Submits execution of Hadoop job to grid.
+     *
+     * @param jobId Job ID.
+     * @param info Job info.
+     * @return Job completion future.
+     */
+    @SuppressWarnings("unchecked")
+    public IgniteInternalFuture<GridHadoopJobId> submit(GridHadoopJobId jobId, GridHadoopJobInfo info) {
+        if (!busyLock.tryReadLock()) {
+            return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to execute map-reduce job " +
+                "(grid is stopping): " + info));
+        }
+
+        try {
+            long jobPrepare = U.currentTimeMillis();
+
+            if (jobs.containsKey(jobId) || jobMetaCache().containsKey(jobId))
+                throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
+
+            GridHadoopJob job = job(jobId, info);
+
+            GridHadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
+
+            GridHadoopJobMetadata meta = new GridHadoopJobMetadata(ctx.localNodeId(), jobId, info);
+
+            meta.mapReducePlan(mrPlan);
+
+            meta.pendingSplits(allSplits(mrPlan));
+            meta.pendingReducers(allReducers(mrPlan));
+
+            GridFutureAdapter<GridHadoopJobId> completeFut = new GridFutureAdapter<>();
+
+            GridFutureAdapter<GridHadoopJobId> old = activeFinishFuts.put(jobId, completeFut);
+
+            assert old == null : "Duplicate completion future [jobId=" + jobId + ", old=" + old + ']';
+
+            if (log.isDebugEnabled())
+                log.debug("Submitting job metadata [jobId=" + jobId + ", meta=" + meta + ']');
+
+            long jobStart = U.currentTimeMillis();
+
+            GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(meta.counters(),
+                ctx.localNodeId());
+
+            perfCntr.clientSubmissionEvents(info);
+            perfCntr.onJobPrepare(jobPrepare);
+            perfCntr.onJobStart(jobStart);
+
+            if (jobMetaCache().putIfAbsent(jobId, meta) != null)
+                throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
+
+            return completeFut;
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to submit job: " + jobId, e);
+
+            return new GridFinishedFutureEx<>(e);
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * Convert Hadoop job metadata to job status.
+     *
+     * @param meta Metadata.
+     * @return Status.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    public static GridHadoopJobStatus status(GridHadoopJobMetadata meta) {
+        GridHadoopJobInfo jobInfo = meta.jobInfo();
+
+        return new GridHadoopJobStatus(
+            meta.jobId(),
+            jobInfo.jobName(),
+            jobInfo.user(),
+            meta.pendingSplits() != null ? meta.pendingSplits().size() : 0,
+            meta.pendingReducers() != null ? meta.pendingReducers().size() : 0,
+            meta.mapReducePlan().mappers(),
+            meta.mapReducePlan().reducers(),
+            meta.phase(),
+            meta.failCause() != null,
+            meta.version()
+        );
+    }
+
+    /**
+     * Gets hadoop job status for given job ID.
+     *
+     * @param jobId Job ID to get status for.
+     * @return Job status for given job ID or {@code null} if job was not found.
+     */
+    @Nullable public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null; // Grid is stopping.
+
+        try {
+            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+            return meta != null ? status(meta) : null;
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * Gets job finish future.
+     *
+     * @param jobId Job ID.
+     * @return Finish future or {@code null}.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null; // Grid is stopping.
+
+        try {
+            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+            if (meta == null)
+                return null;
+
+            if (log.isTraceEnabled())
+                log.trace("Got job metadata for status check [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
+
+            if (meta.phase() == PHASE_COMPLETE) {
+                if (log.isTraceEnabled())
+                    log.trace("Job is complete, returning finished future: " + jobId);
+
+                return new GridFinishedFutureEx<>(jobId, meta.failCause());
+            }
+
+            GridFutureAdapter<GridHadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId,
+                new GridFutureAdapter<GridHadoopJobId>());
+
+            // Get meta from cache one more time to close the window.
+            meta = jobMetaCache().get(jobId);
+
+            if (log.isTraceEnabled())
+                log.trace("Re-checking job metadata [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
+
+            if (meta == null) {
+                fut.onDone();
+
+                activeFinishFuts.remove(jobId , fut);
+            }
+            else if (meta.phase() == PHASE_COMPLETE) {
+                fut.onDone(jobId, meta.failCause());
+
+                activeFinishFuts.remove(jobId , fut);
+            }
+
+            return fut;
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * Gets job plan by job ID.
+     *
+     * @param jobId Job ID.
+     * @return Job plan.
+     * @throws IgniteCheckedException If failed.
+     */
+    public GridHadoopMapReducePlan plan(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null;
+
+        try {
+            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+            if (meta != null)
+                return meta.mapReducePlan();
+
+            return null;
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * Callback from task executor invoked when a task has been finished.
+     *
+     * @param info Task info.
+     * @param status Task status.
+     */
+    @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
+    public void onTaskFinished(GridHadoopTaskInfo info, GridHadoopTaskStatus status) {
+        if (!busyLock.tryReadLock())
+            return;
+
+        try {
+            assert status.state() != RUNNING;
+
+            if (log.isDebugEnabled())
+                log.debug("Received task finished callback [info=" + info + ", status=" + status + ']');
+
+            JobLocalState state = activeJobs.get(info.jobId());
+
+            // Task CRASHes with null fail cause.
+            assert (status.state() != FAILED) || status.failCause() != null :
+                "Invalid task status [info=" + info + ", status=" + status + ']';
+
+            assert state != null || (ctx.jobUpdateLeader() && (info.type() == COMMIT || info.type() == ABORT)):
+                "Missing local state for finished task [info=" + info + ", status=" + status + ']';
+
+            StackedProcessor incrCntrs = null;
+
+            if (status.state() == COMPLETED)
+                incrCntrs = new IncrementCountersProcessor(null, status.counters());
+
+            switch (info.type()) {
+                case SETUP: {
+                    state.onSetupFinished(info, status, incrCntrs);
+
+                    break;
+                }
+
+                case MAP: {
+                    state.onMapFinished(info, status, incrCntrs);
+
+                    break;
+                }
+
+                case REDUCE: {
+                    state.onReduceFinished(info, status, incrCntrs);
+
+                    break;
+                }
+
+                case COMBINE: {
+                    state.onCombineFinished(info, status, incrCntrs);
+
+                    break;
+                }
+
+                case COMMIT:
+                case ABORT: {
+                    GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> cache = finishedJobMetaCache();
+
+                    cache.invokeAsync(info.jobId(), new UpdatePhaseProcessor(incrCntrs, PHASE_COMPLETE)).
+                        listenAsync(failsLog);
+
+                    break;
+                }
+            }
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * @param jobId Job id.
+     * @param c Closure of operation.
+     */
+    private void transform(GridHadoopJobId jobId, EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void> c) {
+        jobMetaCache().invokeAsync(jobId, c).listenAsync(failsLog);
+    }
+
+    /**
+     * Callback from task executor called when process is ready to received shuffle messages.
+     *
+     * @param jobId Job ID.
+     * @param reducers Reducers.
+     * @param desc Process descriptor.
+     */
+    public void onExternalMappersInitialized(GridHadoopJobId jobId, Collection<Integer> reducers,
+        GridHadoopProcessDescriptor desc) {
+        transform(jobId, new InitializeReducersProcessor(null, reducers, desc));
+    }
+
+    /**
+     * Gets all input splits for given hadoop map-reduce plan.
+     *
+     * @param plan Map-reduce plan.
+     * @return Collection of all input splits that should be processed.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private Map<GridHadoopInputSplit, Integer> allSplits(GridHadoopMapReducePlan plan) {
+        Map<GridHadoopInputSplit, Integer> res = new HashMap<>();
+
+        int taskNum = 0;
+
+        for (UUID nodeId : plan.mapperNodeIds()) {
+            for (GridHadoopInputSplit split : plan.mappers(nodeId)) {
+                if (res.put(split, taskNum++) != null)
+                    throw new IllegalStateException("Split duplicate.");
+            }
+        }
+
+        return res;
+    }
+
+    /**
+     * Gets all reducers for this job.
+     *
+     * @param plan Map-reduce plan.
+     * @return Collection of reducers.
+     */
+    private Collection<Integer> allReducers(GridHadoopMapReducePlan plan) {
+        Collection<Integer> res = new HashSet<>();
+
+        for (int i = 0; i < plan.reducers(); i++)
+            res.add(i);
+
+        return res;
+    }
+
+    /**
+     * Processes node leave (or fail) event.
+     *
+     * @param evt Discovery event.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private void processNodeLeft(DiscoveryEvent evt) {
+        if (log.isDebugEnabled())
+            log.debug("Processing discovery event [locNodeId=" + ctx.localNodeId() + ", evt=" + evt + ']');
+
+        // Check only if this node is responsible for job status updates.
+        if (ctx.jobUpdateLeader()) {
+            boolean checkSetup = evt.eventNode().order() < ctx.localNodeOrder();
+
+            // Iteration over all local entries is correct since system cache is REPLICATED.
+            for (Object metaObj : jobMetaCache().values()) {
+                GridHadoopJobMetadata meta = (GridHadoopJobMetadata)metaObj;
+
+                GridHadoopJobId jobId = meta.jobId();
+
+                GridHadoopMapReducePlan plan = meta.mapReducePlan();
+
+                GridHadoopJobPhase phase = meta.phase();
+
+                try {
+                    if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
+                        // Failover setup task.
+                        GridHadoopJob job = job(jobId, meta.jobInfo());
+
+                        Collection<GridHadoopTaskInfo> setupTask = setupTask(jobId);
+
+                        assert setupTask != null;
+
+                        ctx.taskExecutor().run(job, setupTask);
+                    }
+                    else if (phase == PHASE_MAP || phase == PHASE_REDUCE) {
+                        // Must check all nodes, even that are not event node ID due to
+                        // multiple node failure possibility.
+                        Collection<GridHadoopInputSplit> cancelSplits = null;
+
+                        for (UUID nodeId : plan.mapperNodeIds()) {
+                            if (ctx.kernalContext().discovery().node(nodeId) == null) {
+                                // Node has left the grid.
+                                Collection<GridHadoopInputSplit> mappers = plan.mappers(nodeId);
+
+                                if (cancelSplits == null)
+                                    cancelSplits = new HashSet<>();
+
+                                cancelSplits.addAll(mappers);
+                            }
+                        }
+
+                        Collection<Integer> cancelReducers = null;
+
+                        for (UUID nodeId : plan.reducerNodeIds()) {
+                            if (ctx.kernalContext().discovery().node(nodeId) == null) {
+                                // Node has left the grid.
+                                int[] reducers = plan.reducers(nodeId);
+
+                                if (cancelReducers == null)
+                                    cancelReducers = new HashSet<>();
+
+                                for (int rdc : reducers)
+                                    cancelReducers.add(rdc);
+                            }
+                        }
+
+                        if (cancelSplits != null || cancelReducers != null)
+                            jobMetaCache().invoke(meta.jobId(), new CancelJobProcessor(null, new IgniteCheckedException(
+                                "One or more nodes participating in map-reduce job execution failed."), cancelSplits,
+                                cancelReducers));
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to cancel job: " + meta, e);
+                }
+            }
+        }
+    }
+
+    /**
+     * @param updated Updated cache entries.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void processJobMetadataUpdates(
+        Iterable<CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata>> updated)
+        throws IgniteCheckedException {
+        UUID locNodeId = ctx.localNodeId();
+
+        for (CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata> entry : updated) {
+            GridHadoopJobId jobId = entry.getKey();
+            GridHadoopJobMetadata meta = entry.getValue();
+
+            if (meta == null || !ctx.isParticipating(meta))
+                continue;
+
+            if (log.isDebugEnabled())
+                log.debug("Processing job metadata update callback [locNodeId=" + locNodeId +
+                    ", meta=" + meta + ']');
+
+            try {
+                ctx.taskExecutor().onJobStateChanged(meta);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to process job state changed callback (will fail the job) " +
+                    "[locNodeId=" + locNodeId + ", jobId=" + jobId + ", meta=" + meta + ']', e);
+
+                transform(jobId, new CancelJobProcessor(null, e));
+
+                continue;
+            }
+
+            processJobMetaUpdate(jobId, meta, locNodeId);
+        }
+    }
+
+    /**
+     * @param jobId  Job ID.
+     * @param plan Map-reduce plan.
+     */
+    private void printPlan(GridHadoopJobId jobId, GridHadoopMapReducePlan plan) {
+        log.info("Plan for " + jobId);
+
+        SB b = new SB();
+
+        b.a("   Map: ");
+
+        for (UUID nodeId : plan.mapperNodeIds())
+            b.a(nodeId).a("=").a(plan.mappers(nodeId).size()).a(' ');
+
+        log.info(b.toString());
+
+        b = new SB();
+
+        b.a("   Reduce: ");
+
+        for (UUID nodeId : plan.reducerNodeIds())
+            b.a(nodeId).a("=").a(Arrays.toString(plan.reducers(nodeId))).a(' ');
+
+        log.info(b.toString());
+    }
+
+    /**
+     * @param jobId Job ID.
+     * @param meta Job metadata.
+     * @param locNodeId Local node ID.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void processJobMetaUpdate(GridHadoopJobId jobId, GridHadoopJobMetadata meta, UUID locNodeId)
+        throws IgniteCheckedException {
+        JobLocalState state = activeJobs.get(jobId);
+
+        GridHadoopJob job = job(jobId, meta.jobInfo());
+
+        GridHadoopMapReducePlan plan = meta.mapReducePlan();
+
+        switch (meta.phase()) {
+            case PHASE_SETUP: {
+                if (ctx.jobUpdateLeader()) {
+                    Collection<GridHadoopTaskInfo> setupTask = setupTask(jobId);
+
+                    if (setupTask != null)
+                        ctx.taskExecutor().run(job, setupTask);
+                }
+
+                break;
+            }
+
+            case PHASE_MAP: {
+                // Check if we should initiate new task on local node.
+                Collection<GridHadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta);
+
+                if (tasks != null)
+                    ctx.taskExecutor().run(job, tasks);
+
+                break;
+            }
+
+            case PHASE_REDUCE: {
+                if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) {
+                    GridHadoopTaskInfo info = new GridHadoopTaskInfo(COMMIT, jobId, 0, 0, null);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Submitting COMMIT task for execution [locNodeId=" + locNodeId +
+                                ", jobId=" + jobId + ']');
+
+                    ctx.taskExecutor().run(job, Collections.singletonList(info));
+
+                    break;
+                }
+
+                Collection<GridHadoopTaskInfo> tasks = reducerTasks(plan.reducers(locNodeId), job);
+
+                if (tasks != null)
+                    ctx.taskExecutor().run(job, tasks);
+
+                break;
+            }
+
+            case PHASE_CANCELLING: {
+                // Prevent multiple task executor notification.
+                if (state != null && state.onCancel()) {
+                    if (log.isDebugEnabled())
+                        log.debug("Cancelling local task execution for job: " + meta);
+
+                    ctx.taskExecutor().cancelTasks(jobId);
+                }
+
+                if (meta.pendingSplits().isEmpty() && meta.pendingReducers().isEmpty()) {
+                    if (ctx.jobUpdateLeader()) {
+                        if (state == null)
+                            state = initState(jobId);
+
+                        // Prevent running multiple abort tasks.
+                        if (state.onAborted()) {
+                            GridHadoopTaskInfo info = new GridHadoopTaskInfo(ABORT, jobId, 0, 0, null);
+
+                            if (log.isDebugEnabled())
+                                log.debug("Submitting ABORT task for execution [locNodeId=" + locNodeId +
+                                        ", jobId=" + jobId + ']');
+
+                            ctx.taskExecutor().run(job, Collections.singletonList(info));
+                        }
+                    }
+
+                    break;
+                }
+                else {
+                    // Check if there are unscheduled mappers or reducers.
+                    Collection<GridHadoopInputSplit> cancelMappers = new ArrayList<>();
+                    Collection<Integer> cancelReducers = new ArrayList<>();
+
+                    Collection<GridHadoopInputSplit> mappers = plan.mappers(ctx.localNodeId());
+
+                    if (mappers != null) {
+                        for (GridHadoopInputSplit b : mappers) {
+                            if (state == null || !state.mapperScheduled(b))
+                                cancelMappers.add(b);
+                        }
+                    }
+
+                    int[] rdc = plan.reducers(ctx.localNodeId());
+
+                    if (rdc != null) {
+                        for (int r : rdc) {
+                            if (state == null || !state.reducerScheduled(r))
+                                cancelReducers.add(r);
+                        }
+                    }
+
+                    if (!cancelMappers.isEmpty() || !cancelReducers.isEmpty())
+                        transform(jobId, new CancelJobProcessor(null, cancelMappers, cancelReducers));
+                }
+
+                break;
+            }
+
+            case PHASE_COMPLETE: {
+                if (log.isDebugEnabled())
+                    log.debug("Job execution is complete, will remove local state from active jobs " +
+                        "[jobId=" + jobId + ", meta=" + meta + ']');
+
+                if (state != null) {
+                    state = activeJobs.remove(jobId);
+
+                    assert state != null;
+
+                    ctx.shuffle().jobFinished(jobId);
+                }
+
+                GridFutureAdapter<GridHadoopJobId> finishFut = activeFinishFuts.remove(jobId);
+
+                if (finishFut != null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Completing job future [locNodeId=" + locNodeId + ", meta=" + meta + ']');
+
+                    finishFut.onDone(jobId, meta.failCause());
+                }
+
+                if (ctx.jobUpdateLeader())
+                    job.cleanupStagingDirectory();
+
+                jobs.remove(jobId);
+
+                job.dispose(false);
+
+                if (ctx.jobUpdateLeader()) {
+                    ClassLoader ldr = job.getClass().getClassLoader();
+
+                    try {
+                        String statWriterClsName = job.info().property(HadoopUtils.JOB_COUNTER_WRITER_PROPERTY);
+
+                        if (statWriterClsName != null) {
+                            Class<?> cls = ldr.loadClass(statWriterClsName);
+
+                            GridHadoopCounterWriter writer = (GridHadoopCounterWriter)cls.newInstance();
+
+                            GridHadoopCounters cntrs = meta.counters();
+
+                            writer.write(job.info(), jobId, cntrs);
+                        }
+                    }
+                    catch (Exception e) {
+                        log.error("Can't write statistic due to: ", e);
+                    }
+                }
+
+                break;
+            }
+
+            default:
+                throw new IllegalStateException("Unknown phase: " + meta.phase());
+        }
+    }
+
+    /**
+     * Creates setup task based on job information.
+     *
+     * @param jobId Job ID.
+     * @return Setup task wrapped in collection.
+     */
+    @Nullable private Collection<GridHadoopTaskInfo> setupTask(GridHadoopJobId jobId) {
+        if (activeJobs.containsKey(jobId))
+            return null;
+        else {
+            initState(jobId);
+
+            return Collections.singleton(new GridHadoopTaskInfo(SETUP, jobId, 0, 0, null));
+        }
+    }
+
+    /**
+     * Creates mapper tasks based on job information.
+     *
+     * @param mappers Mapper blocks.
+     * @param meta Job metadata.
+     * @return Collection of created task infos or {@code null} if no mapper tasks scheduled for local node.
+     */
+    private Collection<GridHadoopTaskInfo> mapperTasks(Iterable<GridHadoopInputSplit> mappers, GridHadoopJobMetadata meta) {
+        UUID locNodeId = ctx.localNodeId();
+        GridHadoopJobId jobId = meta.jobId();
+
+        JobLocalState state = activeJobs.get(jobId);
+
+        Collection<GridHadoopTaskInfo> tasks = null;
+
+        if (mappers != null) {
+            if (state == null)
+                state = initState(jobId);
+
+            for (GridHadoopInputSplit split : mappers) {
+                if (state.addMapper(split)) {
+                    if (log.isDebugEnabled())
+                        log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId +
+                            ", split=" + split + ']');
+
+                    GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split);
+
+                    if (tasks == null)
+                        tasks = new ArrayList<>();
+
+                    tasks.add(taskInfo);
+                }
+            }
+        }
+
+        return tasks;
+    }
+
+    /**
+     * Creates reducer tasks based on job information.
+     *
+     * @param reducers Reducers (may be {@code null}).
+     * @param job Job instance.
+     * @return Collection of task infos.
+     */
+    private Collection<GridHadoopTaskInfo> reducerTasks(int[] reducers, GridHadoopJob job) {
+        UUID locNodeId = ctx.localNodeId();
+        GridHadoopJobId jobId = job.id();
+
+        JobLocalState state = activeJobs.get(jobId);
+
+        Collection<GridHadoopTaskInfo> tasks = null;
+
+        if (reducers != null) {
+            if (state == null)
+                state = initState(job.id());
+
+            for (int rdc : reducers) {
+                if (state.addReducer(rdc)) {
+                    if (log.isDebugEnabled())
+                        log.debug("Submitting REDUCE task for execution [locNodeId=" + locNodeId +
+                            ", rdc=" + rdc + ']');
+
+                    GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(REDUCE, jobId, rdc, 0, null);
+
+                    if (tasks == null)
+                        tasks = new ArrayList<>();
+
+                    tasks.add(taskInfo);
+                }
+            }
+        }
+
+        return tasks;
+    }
+
+    /**
+     * Initializes local state for given job metadata.
+     *
+     * @param jobId Job ID.
+     * @return Local state.
+     */
+    private JobLocalState initState(GridHadoopJobId jobId) {
+        return F.addIfAbsent(activeJobs, jobId, new JobLocalState());
+    }
+
+    /**
+     * Gets or creates job instance.
+     *
+     * @param jobId Job ID.
+     * @param jobInfo Job info.
+     * @return Job.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public GridHadoopJob job(GridHadoopJobId jobId, @Nullable GridHadoopJobInfo jobInfo) throws IgniteCheckedException {
+        GridFutureAdapterEx<GridHadoopJob> fut = jobs.get(jobId);
+
+        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapterEx<GridHadoopJob>())) != null)
+            return fut.get();
+
+        fut = jobs.get(jobId);
+
+        GridHadoopJob job = null;
+
+        try {
+            if (jobInfo == null) {
+                GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+                if (meta == null)
+                    throw new IgniteCheckedException("Failed to find job metadata for ID: " + jobId);
+
+                jobInfo = meta.jobInfo();
+            }
+
+            job = jobInfo.createJob(jobId, log);
+
+            job.initialize(false, ctx.localNodeId());
+
+            fut.onDone(job);
+
+            return job;
+        }
+        catch (IgniteCheckedException e) {
+            fut.onDone(e);
+
+            jobs.remove(jobId, fut);
+
+            if (job != null) {
+                try {
+                    job.dispose(false);
+                }
+                catch (IgniteCheckedException e0) {
+                    U.error(log, "Failed to dispose job: " + jobId, e0);
+                }
+            }
+
+            throw e;
+        }
+    }
+
+    /**
+     * Kills job.
+     *
+     * @param jobId Job ID.
+     * @return {@code True} if job was killed.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean killJob(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return false; // Grid is stopping.
+
+        try {
+            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+            if (meta != null && meta.phase() != PHASE_COMPLETE && meta.phase() != PHASE_CANCELLING) {
+                HadoopTaskCancelledException err = new HadoopTaskCancelledException("Job cancelled.");
+
+                jobMetaCache().invoke(jobId, new CancelJobProcessor(null, err));
+            }
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+
+        IgniteInternalFuture<?> fut = finishFuture(jobId);
+
+        if (fut != null) {
+            try {
+                fut.get();
+            }
+            catch (Throwable e) {
+                if (e.getCause() instanceof HadoopTaskCancelledException)
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Returns job counters.
+     *
+     * @param jobId Job identifier.
+     * @return Job counters or {@code null} if job cannot be found.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public GridHadoopCounters jobCounters(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock())
+            return null;
+
+        try {
+            final GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+
+            return meta != null ? meta.counters() : null;
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /**
+     * Event handler protected by busy lock.
+     */
+    private abstract class EventHandler implements Runnable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            if (!busyLock.tryReadLock())
+                return;
+
+            try {
+                body();
+            }
+            catch (Throwable e) {
+                U.error(log, "Unhandled exception while processing event.", e);
+            }
+            finally {
+                busyLock.readUnlock();
+            }
+        }
+
+        /**
+         * Handler body.
+         */
+        protected abstract void body() throws Exception;
+    }
+
+    /**
+     *
+     */
+    private class JobLocalState {
+        /** Mappers. */
+        private final Collection<GridHadoopInputSplit> currMappers = new HashSet<>();
+
+        /** Reducers. */
+        private final Collection<Integer> currReducers = new HashSet<>();
+
+        /** Number of completed mappers. */
+        private final AtomicInteger completedMappersCnt = new AtomicInteger();
+
+        /** Cancelled flag. */
+        private boolean cancelled;
+
+        /** Aborted flag. */
+        private boolean aborted;
+
+        /**
+         * @param mapSplit Map split to add.
+         * @return {@code True} if mapper was added.
+         */
+        private boolean addMapper(GridHadoopInputSplit mapSplit) {
+            return currMappers.add(mapSplit);
+        }
+
+        /**
+         * @param rdc Reducer number to add.
+         * @return {@code True} if reducer was added.
+         */
+        private boolean addReducer(int rdc) {
+            return currReducers.add(rdc);
+        }
+
+        /**
+         * Checks whether this split was scheduled for given attempt.
+         *
+         * @param mapSplit Map split to check.
+         * @return {@code True} if mapper was scheduled.
+         */
+        public boolean mapperScheduled(GridHadoopInputSplit mapSplit) {
+            return currMappers.contains(mapSplit);
+        }
+
+        /**
+         * Checks whether this split was scheduled for given attempt.
+         *
+         * @param rdc Reducer number to check.
+         * @return {@code True} if reducer was scheduled.
+         */
+        public boolean reducerScheduled(int rdc) {
+            return currReducers.contains(rdc);
+        }
+
+        /**
+         * @param taskInfo Task info.
+         * @param status Task status.
+         * @param prev Previous closure.
+         */
+        private void onSetupFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
+            final GridHadoopJobId jobId = taskInfo.jobId();
+
+            if (status.state() == FAILED || status.state() == CRASHED)
+                transform(jobId, new CancelJobProcessor(prev, status.failCause()));
+            else
+                transform(jobId, new UpdatePhaseProcessor(prev, PHASE_MAP));
+        }
+
+        /**
+         * @param taskInfo Task info.
+         * @param status Task status.
+         * @param prev Previous closure.
+         */
+        private void onMapFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
+            final StackedProcessor prev) {
+            final GridHadoopJobId jobId = taskInfo.jobId();
+
+            boolean lastMapperFinished = completedMappersCnt.incrementAndGet() == currMappers.size();
+
+            if (status.state() == FAILED || status.state() == CRASHED) {
+                // Fail the whole job.
+                transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), status.failCause()));
+
+                return;
+            }
+
+            IgniteInClosure<IgniteInternalFuture<?>> cacheUpdater = new CIX1<IgniteInternalFuture<?>>() {
+                @Override public void applyx(IgniteInternalFuture<?> f) {
+                    Throwable err = null;
+
+                    if (f != null) {
+                        try {
+                            f.get();
+                        }
+                        catch (IgniteCheckedException e) {
+                            err = e;
+                        }
+                    }
+
+                    transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), err));
+                }
+            };
+
+            if (lastMapperFinished)
+                ctx.shuffle().flush(jobId).listenAsync(cacheUpdater);
+            else
+                cacheUpdater.apply(null);
+        }
+
+        /**
+         * @param taskInfo Task info.
+         * @param status Task status.
+         * @param prev Previous closure.
+         */
+        private void onReduceFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
+            GridHadoopJobId jobId = taskInfo.jobId();
+            if (status.state() == FAILED || status.state() == CRASHED)
+                // Fail the whole job.
+                transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber(), status.failCause()));
+            else
+                transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber()));
+        }
+
+        /**
+         * @param taskInfo Task info.
+         * @param status Task status.
+         * @param prev Previous closure.
+         */
+        private void onCombineFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
+            final StackedProcessor prev) {
+            final GridHadoopJobId jobId = taskInfo.jobId();
+
+            if (status.state() == FAILED || status.state() == CRASHED)
+                // Fail the whole job.
+                transform(jobId, new RemoveMappersProcessor(prev, currMappers, status.failCause()));
+            else {
+                ctx.shuffle().flush(jobId).listenAsync(new CIX1<IgniteInternalFuture<?>>() {
+                    @Override public void applyx(IgniteInternalFuture<?> f) {
+                        Throwable err = null;
+
+                        if (f != null) {
+                            try {
+                                f.get();
+                            }
+                            catch (IgniteCheckedException e) {
+                                err = e;
+                            }
+                        }
+
+                        transform(jobId, new RemoveMappersProcessor(prev, currMappers, err));
+                    }
+                });
+            }
+        }
+
+        /**
+         * @return {@code True} if job was cancelled by this (first) call.
+         */
+        public boolean onCancel() {
+            if (!cancelled && !aborted) {
+                cancelled = true;
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /**
+         * @return {@code True} if job was aborted this (first) call.
+         */
+        public boolean onAborted() {
+            if (!aborted) {
+                aborted = true;
+
+                return true;
+            }
+
+            return false;
+        }
+    }
+
+    /**
+     * Update job phase transform closure.
+     */
+    private static class UpdatePhaseProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Phase to update. */
+        private final GridHadoopJobPhase phase;
+
+        /**
+         * @param prev Previous closure.
+         * @param phase Phase to update.
+         */
+        private UpdatePhaseProcessor(@Nullable StackedProcessor prev, GridHadoopJobPhase phase) {
+            super(prev);
+
+            this.phase = phase;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            cp.phase(phase);
+        }
+    }
+
+    /**
+     * Remove mapper transform closure.
+     */
+    private static class RemoveMappersProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Mapper split to remove. */
+        private final Collection<GridHadoopInputSplit> splits;
+
+        /** Error. */
+        private final Throwable err;
+
+        /**
+         * @param prev Previous closure.
+         * @param split Mapper split to remove.
+         * @param err Error.
+         */
+        private RemoveMappersProcessor(@Nullable StackedProcessor prev, GridHadoopInputSplit split, Throwable err) {
+            this(prev, Collections.singletonList(split), err);
+        }
+
+        /**
+         * @param prev Previous closure.
+         * @param splits Mapper splits to remove.
+         * @param err Error.
+         */
+        private RemoveMappersProcessor(@Nullable StackedProcessor prev, Collection<GridHadoopInputSplit> splits,
+            Throwable err) {
+            super(prev);
+
+            this.splits = splits;
+            this.err = err;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            Map<GridHadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+
+            for (GridHadoopInputSplit s : splits)
+                splitsCp.remove(s);
+
+            cp.pendingSplits(splitsCp);
+
+            if (cp.phase() != PHASE_CANCELLING && err != null)
+                cp.failCause(err);
+
+            if (err != null)
+                cp.phase(PHASE_CANCELLING);
+
+            if (splitsCp.isEmpty()) {
+                if (cp.phase() != PHASE_CANCELLING)
+                    cp.phase(PHASE_REDUCE);
+            }
+        }
+    }
+
+    /**
+     * Remove reducer transform closure.
+     */
+    private static class RemoveReducerProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Mapper split to remove. */
+        private final int rdc;
+
+        /** Error. */
+        private Throwable err;
+
+        /**
+         * @param prev Previous closure.
+         * @param rdc Reducer to remove.
+         */
+        private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc) {
+            super(prev);
+
+            this.rdc = rdc;
+        }
+
+        /**
+         * @param prev Previous closure.
+         * @param rdc Reducer to remove.
+         * @param err Error.
+         */
+        private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc, Throwable err) {
+            super(prev);
+
+            this.rdc = rdc;
+            this.err = err;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
+
+            rdcCp.remove(rdc);
+
+            cp.pendingReducers(rdcCp);
+
+            if (err != null) {
+                cp.phase(PHASE_CANCELLING);
+                cp.failCause(err);
+            }
+        }
+    }
+
+    /**
+     * Initialize reducers.
+     */
+    private static class InitializeReducersProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Reducers. */
+        private final Collection<Integer> rdc;
+
+        /** Process descriptor for reducers. */
+        private final GridHadoopProcessDescriptor desc;
+
+        /**
+         * @param prev Previous closure.
+         * @param rdc Reducers to initialize.
+         * @param desc External process descriptor.
+         */
+        private InitializeReducersProcessor(@Nullable StackedProcessor prev,
+            Collection<Integer> rdc,
+            GridHadoopProcessDescriptor desc) {
+            super(prev);
+
+            assert !F.isEmpty(rdc);
+            assert desc != null;
+
+            this.rdc = rdc;
+            this.desc = desc;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            Map<Integer, GridHadoopProcessDescriptor> oldMap = meta.reducersAddresses();
+
+            Map<Integer, GridHadoopProcessDescriptor> rdcMap = oldMap == null ?
+                new HashMap<Integer, GridHadoopProcessDescriptor>() : new HashMap<>(oldMap);
+
+            for (Integer r : rdc)
+                rdcMap.put(r, desc);
+
+            cp.reducersAddresses(rdcMap);
+        }
+    }
+
+    /**
+     * Remove reducer transform closure.
+     */
+    private static class CancelJobProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Mapper split to remove. */
+        private final Collection<GridHadoopInputSplit> splits;
+
+        /** Reducers to remove. */
+        private final Collection<Integer> rdc;
+
+        /** Error. */
+        private final Throwable err;
+
+        /**
+         * @param prev Previous closure.
+         * @param err Fail cause.
+         */
+        private CancelJobProcessor(@Nullable StackedProcessor prev, Throwable err) {
+            this(prev, err, null, null);
+        }
+
+        /**
+         * @param prev Previous closure.
+         * @param splits Splits to remove.
+         * @param rdc Reducers to remove.
+         */
+        private CancelJobProcessor(@Nullable StackedProcessor prev,
+            Collection<GridHadoopInputSplit> splits,
+            Collection<Integer> rdc) {
+            this(prev, null, splits, rdc);
+        }
+
+        /**
+         * @param prev Previous closure.
+         * @param err Error.
+         * @param splits Splits to remove.
+         * @param rdc Reducers to remove.
+         */
+        private CancelJobProcessor(@Nullable StackedProcessor prev,
+            Throwable err,
+            Collection<GridHadoopInputSplit> splits,
+            Collection<Integer> rdc) {
+            super(prev);
+
+            this.splits = splits;
+            this.rdc = rdc;
+            this.err = err;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            assert meta.phase() == PHASE_CANCELLING || err != null: "Invalid phase for cancel: " + meta;
+
+            Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
+
+            if (rdc != null)
+                rdcCp.removeAll(rdc);
+
+            cp.pendingReducers(rdcCp);
+
+            Map<GridHadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
+
+            if (splits != null) {
+                for (GridHadoopInputSplit s : splits)
+                    splitsCp.remove(s);
+            }
+
+            cp.pendingSplits(splitsCp);
+
+            cp.phase(PHASE_CANCELLING);
+
+            if (err != null)
+                cp.failCause(err);
+        }
+    }
+
+    /**
+     * Increment counter values closure.
+     */
+    private static class IncrementCountersProcessor extends StackedProcessor {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final GridHadoopCounters counters;
+
+        /**
+         * @param prev Previous closure.
+         * @param counters Task counters to add into job counters.
+         */
+        private IncrementCountersProcessor(@Nullable StackedProcessor prev, GridHadoopCounters counters) {
+            super(prev);
+
+            assert counters != null;
+
+            this.counters = counters;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+            GridHadoopCounters cntrs = new GridHadoopCountersImpl(cp.counters());
+
+            cntrs.merge(counters);
+
+            cp.counters(cntrs);
+        }
+    }
+
+    /**
+     * Abstract stacked closure.
+     */
+    private abstract static class StackedProcessor implements
+        EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void>, Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final StackedProcessor prev;
+
+        /**
+         * @param prev Previous closure.
+         */
+        private StackedProcessor(@Nullable StackedProcessor prev) {
+            this.prev = prev;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Void process(MutableEntry<GridHadoopJobId, GridHadoopJobMetadata> e, Object... args) {
+            GridHadoopJobMetadata val = apply(e.getValue());
+
+            if (val != null)
+                e.setValue(val);
+            else
+                e.remove();;
+
+            return null;
+        }
+
+        /**
+         * @param meta Old value.
+         * @return New value.
+         */
+        private GridHadoopJobMetadata apply(GridHadoopJobMetadata meta) {
+            if (meta == null)
+                return null;
+
+            GridHadoopJobMetadata cp = prev != null ? prev.apply(meta) : new GridHadoopJobMetadata(meta);
+
+            update(meta, cp);
+
+            return cp;
+        }
+
+        /**
+         * Update given job metadata object.
+         *
+         * @param meta Initial job metadata.
+         * @param cp Copy.
+         */
+        protected abstract void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
index c734acd..8fdab9d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
@@ -37,7 +37,7 @@ public class GridHadoopProtocolSubmitJobTask extends GridHadoopProtocolTaskAdapt
         GridHadoopProtocolTaskArguments args) throws IgniteCheckedException {
         UUID nodeId = UUID.fromString(args.<String>get(0));
         Integer id = args.get(1);
-        GridHadoopDefaultJobInfo info = args.get(2);
+        HadoopDefaultJobInfo info = args.get(2);
 
         assert nodeId != null;
         assert id != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
index 4c83ace..66fb230 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.io.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Hadoop client protocol.
@@ -238,7 +238,7 @@ public class HadoopClientProtocol implements ClientProtocol {
     @Override public String getStagingAreaDir() throws IOException, InterruptedException {
         String usr = UserGroupInformation.getCurrentUser().getShortUserName();
 
-        return GridHadoopUtils.stagingAreaDir(conf, usr).toString();
+        return HadoopUtils.stagingAreaDir(conf, usr).toString();
     }
 
     /** {@inheritDoc} */
@@ -327,6 +327,6 @@ public class HadoopClientProtocol implements ClientProtocol {
         else
             assert lastStatus != null;
 
-        return GridHadoopUtils.status(lastStatus, conf);
+        return HadoopUtils.status(lastStatus, conf);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
deleted file mode 100644
index 396124e..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
+++ /dev/null
@@ -1,256 +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.shuffle;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Shuffle.
- */
-public class GridHadoopShuffle extends GridHadoopComponent {
-    /** */
-    private final ConcurrentMap<GridHadoopJobId, GridHadoopShuffleJob<UUID>> jobs = new ConcurrentHashMap<>();
-
-    /** */
-    protected final GridUnsafeMemory mem = new GridUnsafeMemory(0);
-
-    /** {@inheritDoc} */
-    @Override public void start(GridHadoopContext ctx) throws IgniteCheckedException {
-        super.start(ctx);
-
-        ctx.kernalContext().io().addUserMessageListener(GridTopic.TOPIC_HADOOP,
-            new IgniteBiPredicate<UUID, Object>() {
-                @Override public boolean apply(UUID nodeId, Object msg) {
-                    return onMessageReceived(nodeId, (GridHadoopMessage)msg);
-                }
-            });
-    }
-
-    /**
-     * Stops shuffle.
-     *
-     * @param cancel If should cancel all ongoing activities.
-     */
-    @Override public void stop(boolean cancel) {
-        for (GridHadoopShuffleJob job : jobs.values()) {
-            try {
-                job.close();
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to close job.", e);
-            }
-        }
-
-        jobs.clear();
-    }
-
-    /**
-     * Creates new shuffle job.
-     *
-     * @param jobId Job ID.
-     * @return Created shuffle job.
-     * @throws IgniteCheckedException If job creation failed.
-     */
-    private GridHadoopShuffleJob<UUID> newJob(GridHadoopJobId jobId) throws IgniteCheckedException {
-        GridHadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
-
-        GridHadoopShuffleJob<UUID> job = new GridHadoopShuffleJob<>(ctx.localNodeId(), log,
-            ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId()));
-
-        UUID[] rdcAddrs = new UUID[plan.reducers()];
-
-        for (int i = 0; i < rdcAddrs.length; i++) {
-            UUID nodeId = plan.nodeForReducer(i);
-
-            assert nodeId != null : "Plan is missing node for reducer [plan=" + plan + ", rdc=" + i + ']';
-
-            rdcAddrs[i] = nodeId;
-        }
-
-        boolean init = job.initializeReduceAddresses(rdcAddrs);
-
-        assert init;
-
-        return job;
-    }
-
-    /**
-     * @param nodeId Node ID to send message to.
-     * @param msg Message to send.
-     * @throws IgniteCheckedException If send failed.
-     */
-    private void send0(UUID nodeId, Object msg) throws IgniteCheckedException {
-        ClusterNode node = ctx.kernalContext().discovery().node(nodeId);
-
-        ctx.kernalContext().io().sendUserMessage(F.asList(node), msg, GridTopic.TOPIC_HADOOP, false, 0);
-    }
-
-    /**
-     * @param jobId Task info.
-     * @return Shuffle job.
-     */
-    private GridHadoopShuffleJob<UUID> job(GridHadoopJobId jobId) throws IgniteCheckedException {
-        GridHadoopShuffleJob<UUID> res = jobs.get(jobId);
-
-        if (res == null) {
-            res = newJob(jobId);
-
-            GridHadoopShuffleJob<UUID> old = jobs.putIfAbsent(jobId, res);
-
-            if (old != null) {
-                res.close();
-
-                res = old;
-            }
-            else if (res.reducersInitialized())
-                startSending(res);
-        }
-
-        return res;
-    }
-
-    /**
-     * Starts message sending thread.
-     *
-     * @param shuffleJob Job to start sending for.
-     */
-    private void startSending(GridHadoopShuffleJob<UUID> shuffleJob) {
-        shuffleJob.startSending(ctx.kernalContext().gridName(),
-            new IgniteInClosure2X<UUID, GridHadoopShuffleMessage>() {
-                @Override public void applyx(UUID dest, GridHadoopShuffleMessage msg) throws IgniteCheckedException {
-                    send0(dest, msg);
-                }
-            }
-        );
-    }
-
-    /**
-     * Message received callback.
-     *
-     * @param src Sender node ID.
-     * @param msg Received message.
-     * @return {@code True}.
-     */
-    public boolean onMessageReceived(UUID src, GridHadoopMessage msg) {
-        if (msg instanceof GridHadoopShuffleMessage) {
-            GridHadoopShuffleMessage m = (GridHadoopShuffleMessage)msg;
-
-            try {
-                job(m.jobId()).onShuffleMessage(m);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Message handling failed.", e);
-            }
-
-            try {
-                // Reply with ack.
-                send0(src, new GridHadoopShuffleAck(m.id(), m.jobId()));
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to reply back to shuffle message sender [snd=" + src + ", msg=" + msg + ']', e);
-            }
-        }
-        else if (msg instanceof GridHadoopShuffleAck) {
-            GridHadoopShuffleAck m = (GridHadoopShuffleAck)msg;
-
-            try {
-                job(m.jobId()).onShuffleAck(m);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Message handling failed.", e);
-            }
-        }
-        else
-            throw new IllegalStateException("Unknown message type received to Hadoop shuffle [src=" + src +
-                ", msg=" + msg + ']');
-
-        return true;
-    }
-
-    /**
-     * @param taskCtx Task info.
-     * @return Output.
-     */
-    public GridHadoopTaskOutput output(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        return job(taskCtx.taskInfo().jobId()).output(taskCtx);
-    }
-
-    /**
-     * @param taskCtx Task info.
-     * @return Input.
-     */
-    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        return job(taskCtx.taskInfo().jobId()).input(taskCtx);
-    }
-
-    /**
-     * @param jobId Job id.
-     */
-    public void jobFinished(GridHadoopJobId jobId) {
-        GridHadoopShuffleJob job = jobs.remove(jobId);
-
-        if (job != null) {
-            try {
-                job.close();
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to close job: " + jobId, e);
-            }
-        }
-    }
-
-    /**
-     * Flushes all the outputs for the given job to remote nodes.
-     *
-     * @param jobId Job ID.
-     * @return Future.
-     */
-    public IgniteInternalFuture<?> flush(GridHadoopJobId jobId) {
-        GridHadoopShuffleJob job = jobs.get(jobId);
-
-        if (job == null)
-            return new GridFinishedFutureEx<>();
-
-        try {
-            return job.flush();
-        }
-        catch (IgniteCheckedException e) {
-            return new GridFinishedFutureEx<>(e);
-        }
-    }
-
-    /**
-     * @return Memory.
-     */
-    public GridUnsafeMemory memory() {
-        return mem;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
new file mode 100644
index 0000000..9880093
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -0,0 +1,256 @@
+/*
+ * 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.shuffle;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Shuffle.
+ */
+public class HadoopShuffle extends HadoopComponent {
+    /** */
+    private final ConcurrentMap<GridHadoopJobId, GridHadoopShuffleJob<UUID>> jobs = new ConcurrentHashMap<>();
+
+    /** */
+    protected final GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+    /** {@inheritDoc} */
+    @Override public void start(HadoopContext ctx) throws IgniteCheckedException {
+        super.start(ctx);
+
+        ctx.kernalContext().io().addUserMessageListener(GridTopic.TOPIC_HADOOP,
+            new IgniteBiPredicate<UUID, Object>() {
+                @Override public boolean apply(UUID nodeId, Object msg) {
+                    return onMessageReceived(nodeId, (GridHadoopMessage)msg);
+                }
+            });
+    }
+
+    /**
+     * Stops shuffle.
+     *
+     * @param cancel If should cancel all ongoing activities.
+     */
+    @Override public void stop(boolean cancel) {
+        for (GridHadoopShuffleJob job : jobs.values()) {
+            try {
+                job.close();
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to close job.", e);
+            }
+        }
+
+        jobs.clear();
+    }
+
+    /**
+     * Creates new shuffle job.
+     *
+     * @param jobId Job ID.
+     * @return Created shuffle job.
+     * @throws IgniteCheckedException If job creation failed.
+     */
+    private GridHadoopShuffleJob<UUID> newJob(GridHadoopJobId jobId) throws IgniteCheckedException {
+        GridHadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
+
+        GridHadoopShuffleJob<UUID> job = new GridHadoopShuffleJob<>(ctx.localNodeId(), log,
+            ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId()));
+
+        UUID[] rdcAddrs = new UUID[plan.reducers()];
+
+        for (int i = 0; i < rdcAddrs.length; i++) {
+            UUID nodeId = plan.nodeForReducer(i);
+
+            assert nodeId != null : "Plan is missing node for reducer [plan=" + plan + ", rdc=" + i + ']';
+
+            rdcAddrs[i] = nodeId;
+        }
+
+        boolean init = job.initializeReduceAddresses(rdcAddrs);
+
+        assert init;
+
+        return job;
+    }
+
+    /**
+     * @param nodeId Node ID to send message to.
+     * @param msg Message to send.
+     * @throws IgniteCheckedException If send failed.
+     */
+    private void send0(UUID nodeId, Object msg) throws IgniteCheckedException {
+        ClusterNode node = ctx.kernalContext().discovery().node(nodeId);
+
+        ctx.kernalContext().io().sendUserMessage(F.asList(node), msg, GridTopic.TOPIC_HADOOP, false, 0);
+    }
+
+    /**
+     * @param jobId Task info.
+     * @return Shuffle job.
+     */
+    private GridHadoopShuffleJob<UUID> job(GridHadoopJobId jobId) throws IgniteCheckedException {
+        GridHadoopShuffleJob<UUID> res = jobs.get(jobId);
+
+        if (res == null) {
+            res = newJob(jobId);
+
+            GridHadoopShuffleJob<UUID> old = jobs.putIfAbsent(jobId, res);
+
+            if (old != null) {
+                res.close();
+
+                res = old;
+            }
+            else if (res.reducersInitialized())
+                startSending(res);
+        }
+
+        return res;
+    }
+
+    /**
+     * Starts message sending thread.
+     *
+     * @param shuffleJob Job to start sending for.
+     */
+    private void startSending(GridHadoopShuffleJob<UUID> shuffleJob) {
+        shuffleJob.startSending(ctx.kernalContext().gridName(),
+            new IgniteInClosure2X<UUID, GridHadoopShuffleMessage>() {
+                @Override public void applyx(UUID dest, GridHadoopShuffleMessage msg) throws IgniteCheckedException {
+                    send0(dest, msg);
+                }
+            }
+        );
+    }
+
+    /**
+     * Message received callback.
+     *
+     * @param src Sender node ID.
+     * @param msg Received message.
+     * @return {@code True}.
+     */
+    public boolean onMessageReceived(UUID src, GridHadoopMessage msg) {
+        if (msg instanceof GridHadoopShuffleMessage) {
+            GridHadoopShuffleMessage m = (GridHadoopShuffleMessage)msg;
+
+            try {
+                job(m.jobId()).onShuffleMessage(m);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Message handling failed.", e);
+            }
+
+            try {
+                // Reply with ack.
+                send0(src, new GridHadoopShuffleAck(m.id(), m.jobId()));
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to reply back to shuffle message sender [snd=" + src + ", msg=" + msg + ']', e);
+            }
+        }
+        else if (msg instanceof GridHadoopShuffleAck) {
+            GridHadoopShuffleAck m = (GridHadoopShuffleAck)msg;
+
+            try {
+                job(m.jobId()).onShuffleAck(m);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Message handling failed.", e);
+            }
+        }
+        else
+            throw new IllegalStateException("Unknown message type received to Hadoop shuffle [src=" + src +
+                ", msg=" + msg + ']');
+
+        return true;
+    }
+
+    /**
+     * @param taskCtx Task info.
+     * @return Output.
+     */
+    public GridHadoopTaskOutput output(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        return job(taskCtx.taskInfo().jobId()).output(taskCtx);
+    }
+
+    /**
+     * @param taskCtx Task info.
+     * @return Input.
+     */
+    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        return job(taskCtx.taskInfo().jobId()).input(taskCtx);
+    }
+
+    /**
+     * @param jobId Job id.
+     */
+    public void jobFinished(GridHadoopJobId jobId) {
+        GridHadoopShuffleJob job = jobs.remove(jobId);
+
+        if (job != null) {
+            try {
+                job.close();
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to close job: " + jobId, e);
+            }
+        }
+    }
+
+    /**
+     * Flushes all the outputs for the given job to remote nodes.
+     *
+     * @param jobId Job ID.
+     * @return Future.
+     */
+    public IgniteInternalFuture<?> flush(GridHadoopJobId jobId) {
+        GridHadoopShuffleJob job = jobs.get(jobId);
+
+        if (job == null)
+            return new GridFinishedFutureEx<>();
+
+        try {
+            return job.flush();
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFutureEx<>(e);
+        }
+    }
+
+    /**
+     * @return Memory.
+     */
+    public GridUnsafeMemory memory() {
+        return mem;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopEmbeddedTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopEmbeddedTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopEmbeddedTaskExecutor.java
deleted file mode 100644
index fde5400..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopEmbeddedTaskExecutor.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.hadoop.taskexecutor;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-
-/**
- * Task executor.
- */
-public class GridHadoopEmbeddedTaskExecutor extends GridHadoopTaskExecutorAdapter {
-    /** Job tracker. */
-    private GridHadoopJobTracker jobTracker;
-
-    /** */
-    private final ConcurrentMap<GridHadoopJobId, Collection<GridHadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
-
-    /** Executor service to run tasks. */
-    private GridHadoopExecutorService exec;
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
-
-        jobTracker = ctx.jobTracker();
-
-        exec = new GridHadoopExecutorService(log, ctx.kernalContext().gridName(),
-            ctx.configuration().getMaxParallelTasks(), ctx.configuration().getMaxTaskQueueSize());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        if (exec != null) {
-            exec.shutdown(3000);
-
-            if (cancel) {
-                for (GridHadoopJobId jobId : jobs.keySet())
-                    cancelTasks(jobId);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) {
-        if (exec != null && !exec.shutdown(30000))
-            U.warn(log, "Failed to finish running tasks in 30 sec.");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void run(final GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException {
-        if (log.isDebugEnabled())
-            log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() +
-                ", tasksCnt=" + tasks.size() + ']');
-
-        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(job.id());
-
-        if (executedTasks == null) {
-            executedTasks = new GridConcurrentHashSet<>();
-
-            Collection<GridHadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks);
-
-            assert extractedCol == null;
-        }
-
-        final Collection<GridHadoopRunnableTask> finalExecutedTasks = executedTasks;
-
-        for (final GridHadoopTaskInfo info : tasks) {
-            assert info != null;
-
-            GridHadoopRunnableTask task = new GridHadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
-                ctx.localNodeId()) {
-                @Override protected void onTaskFinished(GridHadoopTaskStatus status) {
-                    if (log.isDebugEnabled())
-                        log.debug("Finished task execution [jobId=" + job.id() + ", taskInfo=" + info + ", " +
-                            "waitTime=" + waitTime() + ", execTime=" + executionTime() + ']');
-
-                    finalExecutedTasks.remove(this);
-
-                    jobTracker.onTaskFinished(info, status);
-                }
-
-                @Override protected GridHadoopTaskInput createInput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-                    return ctx.shuffle().input(taskCtx);
-                }
-
-                @Override protected GridHadoopTaskOutput createOutput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-                    return ctx.shuffle().output(taskCtx);
-                }
-            };
-
-            executedTasks.add(task);
-
-            exec.submit(task);
-        }
-    }
-
-    /**
-     * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks
-     * for this job ID.
-     * <p>
-     * It is guaranteed that this method will not be called concurrently with
-     * {@link #run(GridHadoopJob, Collection)} method. No more job submissions will be performed via
-     * {@link #run(GridHadoopJob, Collection)} method for given job ID after this method is called.
-     *
-     * @param jobId Job ID to cancel.
-     */
-    @Override public void cancelTasks(GridHadoopJobId jobId) {
-        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(jobId);
-
-        if (executedTasks != null) {
-            for (GridHadoopRunnableTask task : executedTasks)
-                task.cancel();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException {
-        if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
-            Collection<GridHadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
-
-            assert executedTasks == null || executedTasks.isEmpty();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
index fd4a030..0d49be9 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
@@ -130,7 +130,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
                 }
             }
         }
-        catch (GridHadoopTaskCancelledException ignored) {
+        catch (HadoopTaskCancelledException ignored) {
             state = GridHadoopTaskState.CANCELED;
         }
         catch (Throwable e) {
@@ -163,7 +163,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
      */
     private void runTask(GridHadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
         if (cancelled)
-            throw new GridHadoopTaskCancelledException("Task cancelled.");
+            throw new HadoopTaskCancelledException("Task cancelled.");
 
         try (GridHadoopTaskOutput out = createOutputInternal(ctx);
              GridHadoopTaskInput in = createInputInternal(ctx)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskExecutorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskExecutorAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskExecutorAdapter.java
deleted file mode 100644
index 8f66190..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskExecutorAdapter.java
+++ /dev/null
@@ -1,57 +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.taskexecutor;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
-
-import java.util.*;
-
-/**
- * Common superclass for task executor.
- */
-public abstract class GridHadoopTaskExecutorAdapter extends GridHadoopComponent {
-    /**
-     * Runs tasks.
-     *
-     * @param job Job.
-     * @param tasks Tasks.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract void run(final GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException;
-
-    /**
-     * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks
-     * for this job ID.
-     * <p>
-     * It is guaranteed that this method will not be called concurrently with
-     * {@link #run(GridHadoopJob, Collection)} method. No more job submissions will be performed via
-     * {@link #run(GridHadoopJob, Collection)} method for given job ID after this method is called.
-     *
-     * @param jobId Job ID to cancel.
-     */
-    public abstract void cancelTasks(GridHadoopJobId jobId) throws IgniteCheckedException;
-
-    /**
-     * On job state change callback;
-     *
-     * @param meta Job metadata.
-     */
-    public abstract void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException;
-}


[17/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java
new file mode 100644
index 0000000..0a2af6d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2CleanupTask.java
@@ -0,0 +1,73 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.*;
+
+import java.io.*;
+
+/**
+ * Hadoop cleanup task (commits or aborts job).
+ */
+public class HadoopV2CleanupTask extends HadoopV2Task {
+    /** Abort flag. */
+    private final boolean abort;
+
+    /**
+     * @param taskInfo Task info.
+     * @param abort Abort flag.
+     */
+    public HadoopV2CleanupTask(GridHadoopTaskInfo taskInfo, boolean abort) {
+        super(taskInfo);
+
+        this.abort = abort;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException {
+        JobContextImpl jobCtx = taskCtx.jobContext();
+
+        try {
+            OutputFormat outputFormat = getOutputFormat(jobCtx);
+
+            OutputCommitter committer = outputFormat.getOutputCommitter(hadoopContext());
+
+            if (committer != null) {
+                if (abort)
+                    committer.abortJob(jobCtx, JobStatus.State.FAILED);
+                else
+                    committer.commitJob(jobCtx);
+            }
+        }
+        catch (ClassNotFoundException | IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java
new file mode 100644
index 0000000..a4b5eca
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Context.java
@@ -0,0 +1,230 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.mapreduce.task.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Hadoop context implementation for v2 API. It provides IO operations for hadoop tasks.
+ */
+public class HadoopV2Context extends JobContextImpl implements MapContext, ReduceContext {
+    /** Input reader to overriding of GridHadoopTaskContext input. */
+    private RecordReader reader;
+
+    /** Output writer to overriding of GridHadoopTaskContext output. */
+    private RecordWriter writer;
+
+    /** Output is provided by executor environment. */
+    private final GridHadoopTaskOutput output;
+
+    /** Input is provided by executor environment. */
+    private final GridHadoopTaskInput input;
+
+    /** Unique identifier for a task attempt. */
+    private final TaskAttemptID taskAttemptID;
+
+    /** Indicates that this task is to be cancelled. */
+    private volatile boolean cancelled;
+
+    /** Input split. */
+    private InputSplit inputSplit;
+
+    /** */
+    private final GridHadoopTaskContext ctx;
+
+    /** */
+    private String status;
+
+    /**
+     * @param ctx Context for IO operations.
+     */
+    public HadoopV2Context(HadoopV2TaskContext ctx) {
+        super(ctx.jobConf(), ctx.jobContext().getJobID());
+
+        taskAttemptID = ctx.attemptId();
+
+        conf.set("mapreduce.job.id", taskAttemptID.getJobID().toString());
+        conf.set("mapreduce.task.id", taskAttemptID.getTaskID().toString());
+
+        output = ctx.output();
+        input = ctx.input();
+
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputSplit getInputSplit() {
+        if (inputSplit == null) {
+            GridHadoopInputSplit split = ctx.taskInfo().inputSplit();
+
+            if (split == null)
+                return null;
+
+            if (split instanceof GridHadoopFileBlock) {
+                GridHadoopFileBlock fileBlock = (GridHadoopFileBlock)split;
+
+                inputSplit = new FileSplit(new Path(fileBlock.file()), fileBlock.start(), fileBlock.length(), null);
+            }
+            else if (split instanceof HadoopExternalSplit)
+                throw new UnsupportedOperationException(); // TODO
+            else if (split instanceof HadoopSplitWrapper)
+                inputSplit = (InputSplit) HadoopUtils.unwrapSplit((HadoopSplitWrapper) split);
+            else
+                throw new IllegalStateException();
+        }
+
+        return inputSplit;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nextKeyValue() throws IOException, InterruptedException {
+        if (cancelled)
+            throw new HadoopTaskCancelledException("Task cancelled.");
+
+        return reader.nextKeyValue();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getCurrentKey() throws IOException, InterruptedException {
+        if (reader != null)
+            return reader.getCurrentKey();
+
+        return input.key();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object getCurrentValue() throws IOException, InterruptedException {
+        return reader.getCurrentValue();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void write(Object key, Object val) throws IOException, InterruptedException {
+        if (cancelled)
+            throw new HadoopTaskCancelledException("Task cancelled.");
+
+        if (writer != null)
+            writer.write(key, val);
+        else {
+            try {
+                output.write(key, val);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IOException(e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public OutputCommitter getOutputCommitter() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public TaskAttemptID getTaskAttemptID() {
+        return taskAttemptID;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setStatus(String msg) {
+        status = msg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getStatus() {
+        return status;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getProgress() {
+        return 0.5f; // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counter getCounter(Enum<?> cntrName) {
+        return getCounter(cntrName.getDeclaringClass().getName(), cntrName.name());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counter getCounter(String grpName, String cntrName) {
+        return new HadoopV2Counter(ctx.counter(grpName, cntrName, HadoopLongCounter.class));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void progress() {
+        // No-op.
+    }
+
+    /**
+     * Overrides default input data reader.
+     *
+     * @param reader New reader.
+     */
+    public void reader(RecordReader reader) {
+        this.reader = reader;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean nextKey() throws IOException, InterruptedException {
+        if (cancelled)
+            throw new HadoopTaskCancelledException("Task cancelled.");
+
+        return input.next();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Iterable getValues() throws IOException, InterruptedException {
+        return new Iterable() {
+            @Override public Iterator iterator() {
+                return input.values();
+            }
+        };
+    }
+
+    /**
+     * @return Overridden output data writer.
+     */
+    public RecordWriter writer() {
+        return writer;
+    }
+
+    /**
+     * Overrides default output data writer.
+     *
+     * @param writer New writer.
+     */
+    public void writer(RecordWriter writer) {
+        this.writer = writer;
+    }
+
+    /**
+     * Cancels the task by stop the IO.
+     */
+    public void cancel() {
+        cancelled = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java
new file mode 100644
index 0000000..96ede0d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Counter.java
@@ -0,0 +1,87 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+
+import java.io.*;
+
+/**
+ * Adapter from own counter implementation into Hadoop API Counter od version 2.0.
+ */
+public class HadoopV2Counter implements Counter {
+    /** Delegate. */
+    private final HadoopLongCounter cntr;
+
+    /**
+     * Creates new instance with given delegate.
+     *
+     * @param cntr Internal counter.
+     */
+    public HadoopV2Counter(HadoopLongCounter cntr) {
+        assert cntr != null : "counter must be non-null";
+
+        this.cntr = cntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setDisplayName(String displayName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getName() {
+        return cntr.name();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDisplayName() {
+        return getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getValue() {
+        return cntr.value();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(long val) {
+        cntr.value(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void increment(long incr) {
+        cntr.increment(incr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counter getUnderlyingCounter() {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(DataOutput out) throws IOException {
+        throw new UnsupportedOperationException("not implemented");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFields(DataInput in) throws IOException {
+        throw new UnsupportedOperationException("not implemented");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
new file mode 100644
index 0000000..47535e8
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
@@ -0,0 +1,280 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.split.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
+import org.apache.ignite.internal.processors.hadoop.v1.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jdk8.backport.*;
+
+import java.io.*;
+import java.lang.reflect.*;
+import java.util.*;
+import java.util.Queue;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Hadoop job implementation for v2 API.
+ */
+public class HadoopV2Job implements GridHadoopJob {
+    /** */
+    private final JobConf jobConf;
+
+    /** */
+    private final JobContextImpl jobCtx;
+
+    /** Hadoop job ID. */
+    private final GridHadoopJobId jobId;
+
+    /** Job info. */
+    protected GridHadoopJobInfo jobInfo;
+
+    /** */
+    private final JobID hadoopJobID;
+
+    /** */
+    private final HadoopV2JobResourceManager rsrcMgr;
+
+    /** */
+    private final ConcurrentMap<T2<GridHadoopTaskType, Integer>, GridFutureAdapter<GridHadoopTaskContext>> ctxs =
+        new ConcurrentHashMap8<>();
+
+    /** Pooling task context class and thus class loading environment. */
+    private final Queue<Class<?>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
+
+    /** Local node ID */
+    private UUID locNodeId;
+
+    /** Serialized JobConf. */
+    private volatile byte[] jobConfData;
+
+    /**
+     * @param jobId Job ID.
+     * @param jobInfo Job info.
+     * @param log Logger.
+     */
+    public HadoopV2Job(GridHadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log) {
+        assert jobId != null;
+        assert jobInfo != null;
+
+        this.jobId = jobId;
+        this.jobInfo = jobInfo;
+
+        hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
+
+        HadoopClassLoader clsLdr = (HadoopClassLoader)getClass().getClassLoader();
+
+        // Before create JobConf instance we should set new context class loader.
+        Thread.currentThread().setContextClassLoader(clsLdr);
+
+        jobConf = new JobConf();
+
+        HadoopFileSystemsUtils.setupFileSystems(jobConf);
+
+        Thread.currentThread().setContextClassLoader(null);
+
+        for (Map.Entry<String,String> e : jobInfo.properties().entrySet())
+            jobConf.set(e.getKey(), e.getValue());
+
+        jobCtx = new JobContextImpl(jobConf, hadoopJobID);
+
+        rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobId id() {
+        return jobId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobInfo info() {
+        return jobInfo;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridHadoopInputSplit> input() throws IgniteCheckedException {
+        Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
+
+        try {
+            String jobDirPath = jobConf.get(MRJobConfig.MAPREDUCE_JOB_DIR);
+
+            if (jobDirPath == null) { // Probably job was submitted not by hadoop client.
+                // Assume that we have needed classes and try to generate input splits ourself.
+                if (jobConf.getUseNewMapper())
+                    return HadoopV2Splitter.splitJob(jobCtx);
+                else
+                    return HadoopV1Splitter.splitJob(jobConf);
+            }
+
+            Path jobDir = new Path(jobDirPath);
+
+            try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf)) {
+                JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
+                    jobDir);
+
+                if (F.isEmpty(metaInfos))
+                    throw new IgniteCheckedException("No input splits found.");
+
+                Path splitsFile = JobSubmissionFiles.getJobSplitFile(jobDir);
+
+                try (FSDataInputStream in = fs.open(splitsFile)) {
+                    Collection<GridHadoopInputSplit> res = new ArrayList<>(metaInfos.length);
+
+                    for (JobSplit.TaskSplitMetaInfo metaInfo : metaInfos) {
+                        long off = metaInfo.getStartOffset();
+
+                        String[] hosts = metaInfo.getLocations();
+
+                        in.seek(off);
+
+                        String clsName = Text.readString(in);
+
+                        GridHadoopFileBlock block = HadoopV1Splitter.readFileBlock(clsName, in, hosts);
+
+                        if (block == null)
+                            block = HadoopV2Splitter.readFileBlock(clsName, in, hosts);
+
+                        res.add(block != null ? block : new HadoopExternalSplit(hosts, off));
+                    }
+
+                    return res;
+                }
+            }
+            catch (Throwable e) {
+                throw transformException(e);
+            }
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException {
+        T2<GridHadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
+
+        GridFutureAdapter<GridHadoopTaskContext> fut = ctxs.get(locTaskId);
+
+        if (fut != null)
+            return fut.get();
+
+        GridFutureAdapter<GridHadoopTaskContext> old = ctxs.putIfAbsent(locTaskId, fut = new GridFutureAdapter<>());
+
+        if (old != null)
+            return old.get();
+
+        Class<?> cls = taskCtxClsPool.poll();
+
+        try {
+            if (cls == null) {
+                // If there is no pooled class, then load new one.
+                HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath());
+
+                cls = ldr.loadClass(HadoopV2TaskContext.class.getName());
+            }
+
+            Constructor<?> ctr = cls.getConstructor(GridHadoopTaskInfo.class, GridHadoopJob.class,
+                GridHadoopJobId.class, UUID.class, DataInput.class);
+
+            if (jobConfData == null)
+                synchronized(jobConf) {
+                    if (jobConfData == null) {
+                        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+                        jobConf.write(new DataOutputStream(buf));
+
+                        jobConfData = buf.toByteArray();
+                    }
+                }
+
+            GridHadoopTaskContext res = (GridHadoopTaskContext)ctr.newInstance(info, this, jobId, locNodeId,
+                new DataInputStream(new ByteArrayInputStream(jobConfData)));
+
+            fut.onDone(res);
+
+            return res;
+        }
+        catch (Throwable e) {
+            IgniteCheckedException te = transformException(e);
+
+            fut.onDone(te);
+
+            throw te;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException {
+        this.locNodeId = locNodeId;
+
+        Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
+
+        try {
+            rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(locNodeId, jobId));
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dispose(boolean external) throws IgniteCheckedException {
+        if (rsrcMgr != null && !external) {
+            File jobLocDir = jobLocalDir(locNodeId, jobId);
+
+            if (jobLocDir.exists())
+                U.delete(jobLocDir);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
+        rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
+        GridHadoopTaskContext ctx = ctxs.remove(new T2<>(info.type(), info.taskNumber())).get();
+
+        taskCtxClsPool.offer(ctx.getClass());
+
+        File locDir = taskLocalDir(locNodeId, info);
+
+        if (locDir.exists())
+            U.delete(locDir);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cleanupStagingDirectory() {
+        if (rsrcMgr != null)
+            rsrcMgr.cleanupStagingDirectory();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
new file mode 100644
index 0000000..04481bb
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
@@ -0,0 +1,305 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+import java.nio.file.*;
+import java.util.*;
+
+/**
+ * Provides all resources are needed to the job execution. Downloads the main jar, the configuration and additional
+ * files are needed to be placed on local files system.
+ */
+public class HadoopV2JobResourceManager {
+    /** Hadoop job context. */
+    private final JobContextImpl ctx;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Job ID. */
+    private final GridHadoopJobId jobId;
+
+    /** Class path list. */
+    private URL[] clsPath;
+
+    /** Set of local resources. */
+    private final Collection<File> rsrcSet = new HashSet<>();
+
+    /** Staging directory to delivery job jar and config to the work nodes. */
+    private Path stagingDir;
+
+    /**
+     * Creates new instance.
+     * @param jobId Job ID.
+     * @param ctx Hadoop job context.
+     * @param log Logger.
+     */
+    public HadoopV2JobResourceManager(GridHadoopJobId jobId, JobContextImpl ctx, IgniteLogger log) {
+        this.jobId = jobId;
+        this.ctx = ctx;
+        this.log = log.getLogger(HadoopV2JobResourceManager.class);
+    }
+
+    /**
+     * Set working directory in local file system.
+     *
+     * @param dir Working directory.
+     * @throws IOException If fails.
+     */
+    private void setLocalFSWorkingDirectory(File dir) throws IOException {
+        JobConf cfg = ctx.getJobConf();
+
+        Thread.currentThread().setContextClassLoader(cfg.getClassLoader());
+
+        try {
+            cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
+
+            if(!cfg.getBoolean("fs.file.impl.disable.cache", false))
+                FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath()));
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /**
+     * Prepare job resources. Resolve the classpath list and download it if needed.
+     *
+     * @param download {@code true} If need to download resources.
+     * @param jobLocDir Work directory for the job.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void prepareJobEnvironment(boolean download, File jobLocDir) throws IgniteCheckedException {
+        try {
+            if (jobLocDir.exists())
+                throw new IgniteCheckedException("Local job directory already exists: " + jobLocDir.getAbsolutePath());
+
+            JobConf cfg = ctx.getJobConf();
+
+            String mrDir = cfg.get("mapreduce.job.dir");
+
+            if (mrDir != null) {
+                stagingDir = new Path(new URI(mrDir));
+
+                if (download) {
+                    FileSystem fs = FileSystem.get(stagingDir.toUri(), cfg);
+
+                    if (!fs.exists(stagingDir))
+                        throw new IgniteCheckedException("Failed to find map-reduce submission directory (does not exist): " +
+                            stagingDir);
+
+                    if (!FileUtil.copy(fs, stagingDir, jobLocDir, false, cfg))
+                        throw new IgniteCheckedException("Failed to copy job submission directory contents to local file system " +
+                            "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath() + ", jobId=" + jobId + ']');
+                }
+
+                File jarJobFile = new File(jobLocDir, "job.jar");
+
+                Collection<URL> clsPathUrls = new ArrayList<>();
+
+                clsPathUrls.add(jarJobFile.toURI().toURL());
+
+                rsrcSet.add(jarJobFile);
+                rsrcSet.add(new File(jobLocDir, "job.xml"));
+
+                processFiles(jobLocDir, ctx.getCacheFiles(), download, false, null, MRJobConfig.CACHE_LOCALFILES);
+                processFiles(jobLocDir, ctx.getCacheArchives(), download, true, null, MRJobConfig.CACHE_LOCALARCHIVES);
+                processFiles(jobLocDir, ctx.getFileClassPaths(), download, false, clsPathUrls, null);
+                processFiles(jobLocDir, ctx.getArchiveClassPaths(), download, true, clsPathUrls, null);
+
+                if (!clsPathUrls.isEmpty()) {
+                    clsPath = new URL[clsPathUrls.size()];
+
+                    clsPathUrls.toArray(clsPath);
+                }
+            }
+            else if (!jobLocDir.mkdirs())
+                throw new IgniteCheckedException("Failed to create local job directory: " + jobLocDir.getAbsolutePath());
+
+            setLocalFSWorkingDirectory(jobLocDir);
+        }
+        catch (URISyntaxException | IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Process list of resources.
+     *
+     * @param jobLocDir Job working directory.
+     * @param files Array of {@link java.net.URI} or {@link org.apache.hadoop.fs.Path} to process resources.
+     * @param download {@code true}, if need to download. Process class path only else.
+     * @param extract {@code true}, if need to extract archive.
+     * @param clsPathUrls Collection to add resource as classpath resource.
+     * @param rsrcNameProp Property for resource name array setting.
+     * @throws IOException If failed.
+     */
+    private void processFiles(File jobLocDir, @Nullable Object[] files, boolean download, boolean extract,
+        @Nullable Collection<URL> clsPathUrls, @Nullable String rsrcNameProp) throws IOException {
+        if (F.isEmptyOrNulls(files))
+            return;
+
+        Collection<String> res = new ArrayList<>();
+
+        for (Object pathObj : files) {
+            String locName = null;
+            Path srcPath;
+
+            if (pathObj instanceof URI) {
+                URI uri = (URI)pathObj;
+
+                locName = uri.getFragment();
+
+                srcPath = new Path(uri);
+            }
+            else
+                srcPath = (Path)pathObj;
+
+            if (locName == null)
+                locName = srcPath.getName();
+
+            File dstPath = new File(jobLocDir.getAbsolutePath(), locName);
+
+            res.add(locName);
+
+            rsrcSet.add(dstPath);
+
+            if (clsPathUrls != null)
+                clsPathUrls.add(dstPath.toURI().toURL());
+
+            if (!download)
+                continue;
+
+            JobConf cfg = ctx.getJobConf();
+
+            FileSystem dstFs = FileSystem.getLocal(cfg);
+
+            FileSystem srcFs = srcPath.getFileSystem(cfg);
+
+            if (extract) {
+                File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
+
+                if (!archivesPath.exists() && !archivesPath.mkdir())
+                    throw new IOException("Failed to create directory " +
+                         "[path=" + archivesPath + ", jobId=" + jobId + ']');
+
+                File archiveFile = new File(archivesPath, locName);
+
+                FileUtil.copy(srcFs, srcPath, dstFs, new Path(archiveFile.toString()), false, cfg);
+
+                String archiveNameLC = archiveFile.getName().toLowerCase();
+
+                if (archiveNameLC.endsWith(".jar"))
+                    RunJar.unJar(archiveFile, dstPath);
+                else if (archiveNameLC.endsWith(".zip"))
+                    FileUtil.unZip(archiveFile, dstPath);
+                else if (archiveNameLC.endsWith(".tar.gz") ||
+                    archiveNameLC.endsWith(".tgz") ||
+                    archiveNameLC.endsWith(".tar"))
+                    FileUtil.unTar(archiveFile, dstPath);
+                else
+                    throw new IOException("Cannot unpack archive [path=" + srcPath + ", jobId=" + jobId + ']');
+            }
+            else
+                FileUtil.copy(srcFs, srcPath, dstFs, new Path(dstPath.toString()), false, cfg);
+        }
+
+        if (!res.isEmpty() && rsrcNameProp != null)
+            ctx.getJobConf().setStrings(rsrcNameProp, res.toArray(new String[res.size()]));
+    }
+
+    /**
+     * Prepares working directory for the task.
+     *
+     * <ul>
+     *     <li>Creates working directory.</li>
+     *     <li>Creates symbolic links to all job resources in working directory.</li>
+     * </ul>
+     *
+     * @param path Path to working directory of the task.
+     * @throws IgniteCheckedException If fails.
+     */
+    public void prepareTaskWorkDir(File path) throws IgniteCheckedException {
+        try {
+            if (path.exists())
+                throw new IOException("Task local directory already exists: " + path);
+
+            if (!path.mkdir())
+                throw new IOException("Failed to create directory: " + path);
+
+            for (File resource : rsrcSet) {
+                File symLink = new File(path, resource.getName());
+
+                try {
+                    Files.createSymbolicLink(symLink.toPath(), resource.toPath());
+                }
+                catch (IOException e) {
+                    String msg = "Unable to create symlink \"" + symLink + "\" to \"" + resource + "\".";
+
+                    if (U.isWindows() && e instanceof FileSystemException)
+                        msg += "\n\nAbility to create symbolic links is required!\n" +
+                                "On Windows platform you have to grant permission 'Create symbolic links'\n" +
+                                "to your user or run the Accelerator as Administrator.\n";
+
+                    throw new IOException(msg, e);
+                }
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Unable to prepare local working directory for the task " +
+                 "[jobId=" + jobId + ", path=" + path+ ']', e);
+        }
+    }
+
+    /**
+     * Cleans up job staging directory.
+     */
+    public void cleanupStagingDirectory() {
+        try {
+            if (stagingDir != null)
+                stagingDir.getFileSystem(ctx.getJobConf()).delete(stagingDir, true);
+        }
+        catch (Exception e) {
+            log.error("Failed to remove job staging directory [path=" + stagingDir + ", jobId=" + jobId + ']' , e);
+        }
+    }
+
+    /**
+     * Returns array of class path for current job.
+     *
+     * @return Class path collection.
+     */
+    @Nullable public URL[] classPath() {
+        return clsPath;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java
new file mode 100644
index 0000000..afa203f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2MapTask.java
@@ -0,0 +1,109 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.Mapper;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.hadoop.mapreduce.lib.map.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.*;
+
+/**
+ * Hadoop map task implementation for v2 API.
+ */
+public class HadoopV2MapTask extends HadoopV2Task {
+    /**
+     * @param taskInfo Task info.
+     */
+    public HadoopV2MapTask(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions", "unchecked"})
+    @Override public void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException {
+        GridHadoopInputSplit split = info().inputSplit();
+
+        InputSplit nativeSplit;
+
+        if (split instanceof GridHadoopFileBlock) {
+            GridHadoopFileBlock block = (GridHadoopFileBlock)split;
+
+            nativeSplit = new FileSplit(new Path(block.file().toString()), block.start(), block.length(), null);
+        }
+        else
+            nativeSplit = (InputSplit)taskCtx.getNativeSplit(split);
+
+        assert nativeSplit != null;
+
+        OutputFormat outputFormat = null;
+        Exception err = null;
+
+        JobContextImpl jobCtx = taskCtx.jobContext();
+
+        try {
+            InputFormat inFormat = ReflectionUtils.newInstance(jobCtx.getInputFormatClass(),
+                hadoopContext().getConfiguration());
+
+            RecordReader reader = inFormat.createRecordReader(nativeSplit, hadoopContext());
+
+            reader.initialize(nativeSplit, hadoopContext());
+
+            hadoopContext().reader(reader);
+
+            GridHadoopJobInfo jobInfo = taskCtx.job().info();
+
+            outputFormat = jobInfo.hasCombiner() || jobInfo.hasReducer() ? null : prepareWriter(jobCtx);
+
+            Mapper mapper = ReflectionUtils.newInstance(jobCtx.getMapperClass(), hadoopContext().getConfiguration());
+
+            try {
+                mapper.run(new WrappedMapper().getMapContext(hadoopContext()));
+            }
+            finally {
+                closeWriter();
+            }
+
+            commit(outputFormat);
+        }
+        catch (InterruptedException e) {
+            err = e;
+
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+        catch (Exception e) {
+            err = e;
+
+            throw new IgniteCheckedException(e);
+        }
+        finally {
+            if (err != null)
+                abort(outputFormat);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.java
new file mode 100644
index 0000000..83e713b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Partitioner.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.v2;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+/**
+ * Hadoop partitioner adapter for v2 API.
+ */
+public class HadoopV2Partitioner implements GridHadoopPartitioner {
+    /** Partitioner instance. */
+    private Partitioner<Object, Object> part;
+
+    /**
+     * @param cls Hadoop partitioner class.
+     * @param conf Job configuration.
+     */
+    public HadoopV2Partitioner(Class<? extends Partitioner<?, ?>> cls, Configuration conf) {
+        part = (Partitioner<Object, Object>) ReflectionUtils.newInstance(cls, conf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(Object key, Object val, int parts) {
+        return part.getPartition(key, val, parts);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java
new file mode 100644
index 0000000..66ff542
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2ReduceTask.java
@@ -0,0 +1,88 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.reduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.*;
+
+/**
+ * Hadoop reduce task implementation for v2 API.
+ */
+public class HadoopV2ReduceTask extends HadoopV2Task {
+    /** {@code True} if reduce, {@code false} if combine. */
+    private final boolean reduce;
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     * @param reduce {@code True} if reduce, {@code false} if combine.
+     */
+    public HadoopV2ReduceTask(GridHadoopTaskInfo taskInfo, boolean reduce) {
+        super(taskInfo);
+
+        this.reduce = reduce;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions", "unchecked"})
+    @Override public void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException {
+        OutputFormat outputFormat = null;
+        Exception err = null;
+
+        JobContextImpl jobCtx = taskCtx.jobContext();
+
+        try {
+            outputFormat = reduce || !taskCtx.job().info().hasReducer() ? prepareWriter(jobCtx) : null;
+
+            Reducer reducer = ReflectionUtils.newInstance(reduce ? jobCtx.getReducerClass() : jobCtx.getCombinerClass(),
+                jobCtx.getConfiguration());
+
+            try {
+                reducer.run(new WrappedReducer().getReducerContext(hadoopContext()));
+            }
+            finally {
+                closeWriter();
+            }
+
+            commit(outputFormat);
+        }
+        catch (InterruptedException e) {
+            err = e;
+
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+        catch (Exception e) {
+            err = e;
+
+            throw new IgniteCheckedException(e);
+        }
+        finally {
+            if (err != null)
+                abort(outputFormat);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java
new file mode 100644
index 0000000..d0ac792
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2SetupTask.java
@@ -0,0 +1,66 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.OutputCommitter;
+import org.apache.hadoop.mapreduce.OutputFormat;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.*;
+
+import java.io.*;
+
+/**
+ * Hadoop setup task (prepares job).
+ */
+public class HadoopV2SetupTask extends HadoopV2Task {
+    /**
+     * Constructor.
+     *
+     * @param taskInfo task info.
+     */
+    public HadoopV2SetupTask(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override protected void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException {
+        try {
+            JobContextImpl jobCtx = taskCtx.jobContext();
+
+            OutputFormat outputFormat = getOutputFormat(jobCtx);
+
+            outputFormat.checkOutputSpecs(jobCtx);
+
+            OutputCommitter committer = outputFormat.getOutputCommitter(hadoopContext());
+
+            if (committer != null)
+                committer.setupJob(jobCtx);
+        }
+        catch (ClassNotFoundException | IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java
new file mode 100644
index 0000000..d524994
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Splitter.java
@@ -0,0 +1,105 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Hadoop API v2 splitter.
+ */
+public class HadoopV2Splitter {
+    /** */
+    private static final String[] EMPTY_HOSTS = {};
+
+    /**
+     * @param ctx Job context.
+     * @return Collection of mapped splits.
+     * @throws IgniteCheckedException If mapping failed.
+     */
+    public static Collection<GridHadoopInputSplit> splitJob(JobContext ctx) throws IgniteCheckedException {
+        try {
+            InputFormat<?, ?> format = ReflectionUtils.newInstance(ctx.getInputFormatClass(), ctx.getConfiguration());
+
+            assert format != null;
+
+            List<InputSplit> splits = format.getSplits(ctx);
+
+            Collection<GridHadoopInputSplit> res = new ArrayList<>(splits.size());
+
+            int id = 0;
+
+            for (InputSplit nativeSplit : splits) {
+                if (nativeSplit instanceof FileSplit) {
+                    FileSplit s = (FileSplit)nativeSplit;
+
+                    res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
+                }
+                else
+                    res.add(HadoopUtils.wrapSplit(id, nativeSplit, nativeSplit.getLocations()));
+
+                id++;
+            }
+
+            return res;
+        }
+        catch (IOException | ClassNotFoundException e) {
+            throw new IgniteCheckedException(e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteInterruptedCheckedException(e);
+        }
+    }
+
+    /**
+     * @param clsName Input split class name.
+     * @param in Input stream.
+     * @param hosts Optional hosts.
+     * @return File block or {@code null} if it is not a {@link FileSplit} instance.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static GridHadoopFileBlock readFileBlock(String clsName, DataInput in, @Nullable String[] hosts)
+        throws IgniteCheckedException {
+        if (!FileSplit.class.getName().equals(clsName))
+            return null;
+
+        FileSplit split = new FileSplit();
+
+        try {
+            split.readFields(in);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        if (hosts == null)
+            hosts = EMPTY_HOSTS;
+
+        return new GridHadoopFileBlock(hosts, split.getPath().toUri(), split.getStart(), split.getLength());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java
new file mode 100644
index 0000000..04c76ee
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Task.java
@@ -0,0 +1,181 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Extended Hadoop v2 task.
+ */
+public abstract class HadoopV2Task extends GridHadoopTask {
+    /** Hadoop context. */
+    private HadoopV2Context hadoopCtx;
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     */
+    protected HadoopV2Task(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
+
+        hadoopCtx = new HadoopV2Context(ctx);
+
+        run0(ctx);
+    }
+
+    /**
+     * Internal task routine.
+     *
+     * @param taskCtx Task context.
+     * @throws IgniteCheckedException
+     */
+    protected abstract void run0(HadoopV2TaskContext taskCtx) throws IgniteCheckedException;
+
+    /**
+     * @return hadoop context.
+     */
+    protected HadoopV2Context hadoopContext() {
+        return hadoopCtx;
+    }
+
+    /**
+     * Create and configure an OutputFormat instance.
+     *
+     * @param jobCtx Job context.
+     * @return Instance of OutputFormat is specified in job configuration.
+     * @throws ClassNotFoundException If specified class not found.
+     */
+    protected OutputFormat getOutputFormat(JobContext jobCtx) throws ClassNotFoundException {
+        return ReflectionUtils.newInstance(jobCtx.getOutputFormatClass(), hadoopContext().getConfiguration());
+    }
+
+    /**
+     * Put write into Hadoop context and return associated output format instance.
+     *
+     * @param jobCtx Job context.
+     * @return Output format.
+     * @throws IgniteCheckedException In case of Grid exception.
+     * @throws InterruptedException In case of interrupt.
+     */
+    protected OutputFormat prepareWriter(JobContext jobCtx)
+        throws IgniteCheckedException, InterruptedException {
+        try {
+            OutputFormat outputFormat = getOutputFormat(jobCtx);
+
+            assert outputFormat != null;
+
+            OutputCommitter outCommitter = outputFormat.getOutputCommitter(hadoopCtx);
+
+            if (outCommitter != null)
+                outCommitter.setupTask(hadoopCtx);
+
+            RecordWriter writer = outputFormat.getRecordWriter(hadoopCtx);
+
+            hadoopCtx.writer(writer);
+
+            return outputFormat;
+        }
+        catch (IOException | ClassNotFoundException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Closes writer.
+     *
+     * @throws Exception If fails and logger hasn't been specified.
+     */
+    protected void closeWriter() throws Exception {
+        RecordWriter writer = hadoopCtx.writer();
+
+        if (writer != null)
+            writer.close(hadoopCtx);
+    }
+
+    /**
+     * Setup task.
+     *
+     * @param outputFormat Output format.
+     * @throws IOException In case of IO exception.
+     * @throws InterruptedException In case of interrupt.
+     */
+    protected void setup(@Nullable OutputFormat outputFormat) throws IOException, InterruptedException {
+        if (hadoopCtx.writer() != null) {
+            assert outputFormat != null;
+
+            outputFormat.getOutputCommitter(hadoopCtx).setupTask(hadoopCtx);
+        }
+    }
+
+    /**
+     * Commit task.
+     *
+     * @param outputFormat Output format.
+     * @throws IgniteCheckedException In case of Grid exception.
+     * @throws IOException In case of IO exception.
+     * @throws InterruptedException In case of interrupt.
+     */
+    protected void commit(@Nullable OutputFormat outputFormat) throws IgniteCheckedException, IOException, InterruptedException {
+        if (hadoopCtx.writer() != null) {
+            assert outputFormat != null;
+
+            OutputCommitter outputCommitter = outputFormat.getOutputCommitter(hadoopCtx);
+
+            if (outputCommitter.needsTaskCommit(hadoopCtx))
+                outputCommitter.commitTask(hadoopCtx);
+        }
+    }
+
+    /**
+     * Abort task.
+     *
+     * @param outputFormat Output format.
+     */
+    protected void abort(@Nullable OutputFormat outputFormat) {
+        if (hadoopCtx.writer() != null) {
+            assert outputFormat != null;
+
+            try {
+                outputFormat.getOutputCommitter(hadoopCtx).abortTask(hadoopCtx);
+            }
+            catch (IOException ignore) {
+                // Ignore.
+            }
+            catch (InterruptedException ignore) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        hadoopCtx.cancel();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
new file mode 100644
index 0000000..65f6629
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
@@ -0,0 +1,443 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.serializer.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapred.JobID;
+import org.apache.hadoop.mapred.TaskAttemptID;
+import org.apache.hadoop.mapred.TaskID;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
+import org.apache.ignite.internal.processors.hadoop.v1.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Context for task execution.
+ */
+public class HadoopV2TaskContext extends GridHadoopTaskContext {
+    /** */
+    private static final boolean COMBINE_KEY_GROUPING_SUPPORTED;
+
+    /**
+     * Check for combiner grouping support (available since Hadoop 2.3).
+     */
+    static {
+        boolean ok;
+
+        try {
+            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
+
+            ok = true;
+        }
+        catch (NoSuchMethodException ignore) {
+            ok = false;
+        }
+
+        COMBINE_KEY_GROUPING_SUPPORTED = ok;
+    }
+
+    /** Flag is set if new context-object code is used for running the mapper. */
+    private final boolean useNewMapper;
+
+    /** Flag is set if new context-object code is used for running the reducer. */
+    private final boolean useNewReducer;
+
+    /** Flag is set if new context-object code is used for running the combiner. */
+    private final boolean useNewCombiner;
+
+    /** */
+    private final JobContextImpl jobCtx;
+
+    /** Set if task is to cancelling. */
+    private volatile boolean cancelled;
+
+    /** Current task. */
+    private volatile GridHadoopTask task;
+
+    /** Local node ID */
+    private UUID locNodeId;
+
+    /** Counters for task. */
+    private final GridHadoopCounters cntrs = new HadoopCountersImpl();
+
+    /**
+     * @param taskInfo Task info.
+     * @param job Job.
+     * @param jobId Job ID.
+     * @param locNodeId Local node ID.
+     * @param jobConfDataInput DataInput for read JobConf.
+     */
+    public HadoopV2TaskContext(GridHadoopTaskInfo taskInfo, GridHadoopJob job, GridHadoopJobId jobId,
+        @Nullable UUID locNodeId, DataInput jobConfDataInput) throws IgniteCheckedException {
+        super(taskInfo, job);
+        this.locNodeId = locNodeId;
+
+        // Before create JobConf instance we should set new context class loader.
+        Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
+
+        try {
+            JobConf jobConf = new JobConf();
+
+            try {
+                jobConf.readFields(jobConfDataInput);
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException(e);
+            }
+
+            // For map-reduce jobs prefer local writes.
+            jobConf.setBooleanIfUnset(PARAM_IGFS_PREFER_LOCAL_WRITES, true);
+
+            jobCtx = new JobContextImpl(jobConf, new JobID(jobId.globalId().toString(), jobId.localId()));
+
+            useNewMapper = jobConf.getUseNewMapper();
+            useNewReducer = jobConf.getUseNewReducer();
+            useNewCombiner = jobConf.getCombinerClass() == null;
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends GridHadoopCounter> T counter(String grp, String name, Class<T> cls) {
+        return cntrs.counter(grp, name, cls);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopCounters counters() {
+        return cntrs;
+    }
+
+    /**
+     * Creates appropriate task from current task info.
+     *
+     * @return Task.
+     */
+    private GridHadoopTask createTask() {
+        boolean isAbort = taskInfo().type() == GridHadoopTaskType.ABORT;
+
+        switch (taskInfo().type()) {
+            case SETUP:
+                return useNewMapper ? new HadoopV2SetupTask(taskInfo()) : new HadoopV1SetupTask(taskInfo());
+
+            case MAP:
+                return useNewMapper ? new HadoopV2MapTask(taskInfo()) : new HadoopV1MapTask(taskInfo());
+
+            case REDUCE:
+                return useNewReducer ? new HadoopV2ReduceTask(taskInfo(), true) :
+                    new HadoopV1ReduceTask(taskInfo(), true);
+
+            case COMBINE:
+                return useNewCombiner ? new HadoopV2ReduceTask(taskInfo(), false) :
+                    new HadoopV1ReduceTask(taskInfo(), false);
+
+            case COMMIT:
+            case ABORT:
+                return useNewReducer ? new HadoopV2CleanupTask(taskInfo(), isAbort) :
+                    new HadoopV1CleanupTask(taskInfo(), isAbort);
+
+            default:
+                return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run() throws IgniteCheckedException {
+        try {
+            Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
+
+            try {
+                task = createTask();
+            }
+            catch (Throwable e) {
+                throw transformException(e);
+            }
+
+            if (cancelled)
+                throw new HadoopTaskCancelledException("Task cancelled.");
+
+            try {
+                task.run(this);
+            }
+            catch (Throwable e) {
+                throw transformException(e);
+            }
+        }
+        finally {
+            task = null;
+
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancelled = true;
+
+        GridHadoopTask t = task;
+
+        if (t != null)
+            t.cancel();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareTaskEnvironment() throws IgniteCheckedException {
+        File locDir;
+
+        switch(taskInfo().type()) {
+            case MAP:
+            case REDUCE:
+                job().prepareTaskEnvironment(taskInfo());
+
+                locDir = taskLocalDir(locNodeId, taskInfo());
+
+                break;
+
+            default:
+                locDir = jobLocalDir(locNodeId, taskInfo().jobId());
+        }
+
+        Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
+
+        try {
+            FileSystem fs = FileSystem.get(jobConf());
+
+            HadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
+
+            LocalFileSystem locFs = FileSystem.getLocal(jobConf());
+
+            locFs.setWorkingDirectory(new Path(locDir.getAbsolutePath()));
+        }
+        catch (Throwable e) {
+            throw transformException(e);
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cleanupTaskEnvironment() throws IgniteCheckedException {
+        job().cleanupTaskEnvironment(taskInfo());
+    }
+
+    /**
+     * Creates Hadoop attempt ID.
+     *
+     * @return Attempt ID.
+     */
+    public TaskAttemptID attemptId() {
+        TaskID tid = new TaskID(jobCtx.getJobID(), taskType(taskInfo().type()), taskInfo().taskNumber());
+
+        return new TaskAttemptID(tid, taskInfo().attempt());
+    }
+
+    /**
+     * @param type Task type.
+     * @return Hadoop task type.
+     */
+    private TaskType taskType(GridHadoopTaskType type) {
+        switch (type) {
+            case SETUP:
+                return TaskType.JOB_SETUP;
+            case MAP:
+            case COMBINE:
+                return TaskType.MAP;
+
+            case REDUCE:
+                return TaskType.REDUCE;
+
+            case COMMIT:
+            case ABORT:
+                return TaskType.JOB_CLEANUP;
+
+            default:
+                return null;
+        }
+    }
+
+    /**
+     * Gets job configuration of the task.
+     *
+     * @return Job configuration.
+     */
+    public JobConf jobConf() {
+        return jobCtx.getJobConf();
+    }
+
+    /**
+     * Gets job context of the task.
+     *
+     * @return Job context.
+     */
+    public JobContextImpl jobContext() {
+        return jobCtx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopPartitioner partitioner() throws IgniteCheckedException {
+        Class<?> partClsOld = jobConf().getClass("mapred.partitioner.class", null);
+
+        if (partClsOld != null)
+            return new HadoopV1Partitioner(jobConf().getPartitionerClass(), jobConf());
+
+        try {
+            return new HadoopV2Partitioner(jobCtx.getPartitionerClass(), jobConf());
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * Gets serializer for specified class.
+     *
+     * @param cls Class.
+     * @param jobConf Job configuration.
+     * @return Appropriate serializer.
+     */
+    @SuppressWarnings("unchecked")
+    private GridHadoopSerialization getSerialization(Class<?> cls, Configuration jobConf) throws IgniteCheckedException {
+        A.notNull(cls, "cls");
+
+        SerializationFactory factory = new SerializationFactory(jobConf);
+
+        Serialization<?> serialization = factory.getSerialization(cls);
+
+        if (serialization == null)
+            throw new IgniteCheckedException("Failed to find serialization for: " + cls.getName());
+
+        if (serialization.getClass() == WritableSerialization.class)
+            return new HadoopWritableSerialization((Class<? extends Writable>)cls);
+
+        return new HadoopSerializationWrapper(serialization, cls);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopSerialization keySerialization() throws IgniteCheckedException {
+        return getSerialization(jobCtx.getMapOutputKeyClass(), jobConf());
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopSerialization valueSerialization() throws IgniteCheckedException {
+        return getSerialization(jobCtx.getMapOutputValueClass(), jobConf());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Comparator<Object> sortComparator() {
+        return (Comparator<Object>)jobCtx.getSortComparator();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Comparator<Object> groupComparator() {
+        Comparator<?> res;
+
+        switch (taskInfo().type()) {
+            case COMBINE:
+                res = COMBINE_KEY_GROUPING_SUPPORTED ?
+                    jobContext().getCombinerKeyGroupingComparator() : jobContext().getGroupingComparator();
+
+                break;
+
+            case REDUCE:
+                res = jobContext().getGroupingComparator();
+
+                break;
+
+            default:
+                return null;
+        }
+
+        if (res != null && res.getClass() != sortComparator().getClass())
+            return (Comparator<Object>)res;
+
+        return null;
+    }
+
+    /**
+     * @param split Split.
+     * @return Native Hadoop split.
+     * @throws IgniteCheckedException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    public Object getNativeSplit(GridHadoopInputSplit split) throws IgniteCheckedException {
+        if (split instanceof HadoopExternalSplit)
+            return readExternalSplit((HadoopExternalSplit)split);
+
+        if (split instanceof HadoopSplitWrapper)
+            return unwrapSplit((HadoopSplitWrapper)split);
+
+        throw new IllegalStateException("Unknown split: " + split);
+    }
+
+    /**
+     * @param split External split.
+     * @return Native input split.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private Object readExternalSplit(HadoopExternalSplit split) throws IgniteCheckedException {
+        Path jobDir = new Path(jobConf().get(MRJobConfig.MAPREDUCE_JOB_DIR));
+
+        try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf());
+            FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
+
+            in.seek(split.offset());
+
+            String clsName = Text.readString(in);
+
+            Class<?> cls = jobConf().getClassByName(clsName);
+
+            assert cls != null;
+
+            Serialization serialization = new SerializationFactory(jobConf()).getSerialization(cls);
+
+            Deserializer deserializer = serialization.getDeserializer(cls);
+
+            deserializer.open(in);
+
+            Object res = deserializer.deserialize(null);
+
+            deserializer.close();
+
+            assert res != null;
+
+            return res;
+        }
+        catch (IOException | ClassNotFoundException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java
new file mode 100644
index 0000000..cf47e6f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopWritableSerialization.java
@@ -0,0 +1,74 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.io.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Optimized serialization for Hadoop {@link Writable} types.
+ */
+public class HadoopWritableSerialization implements GridHadoopSerialization {
+    /** */
+    private final Class<? extends Writable> cls;
+
+    /**
+     * @param cls Class.
+     */
+    public HadoopWritableSerialization(Class<? extends Writable> cls) {
+        assert cls != null;
+
+        this.cls = cls;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(DataOutput out, Object obj) throws IgniteCheckedException {
+        assert cls.isAssignableFrom(obj.getClass()) : cls + " " + obj.getClass();
+
+        try {
+            ((Writable)obj).write(out);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object read(DataInput in, @Nullable Object obj) throws IgniteCheckedException {
+        Writable w = obj == null ? U.newInstance(cls) : cls.cast(obj);
+
+        try {
+            w.readFields(in);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        return w;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSerializationWrapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSerializationWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSerializationWrapperSelfTest.java
deleted file mode 100644
index 79b9965..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSerializationWrapperSelfTest.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.hadoop.io.*;
-import org.apache.hadoop.io.serializer.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Test of wrapper of the native serialization.
- */
-public class GridHadoopSerializationWrapperSelfTest extends GridCommonAbstractTest {
-    /**
-     * Tests read/write of IntWritable via native WritableSerialization.
-     * @throws Exception If fails.
-     */
-    public void testIntWritableSerialization() throws Exception {
-        GridHadoopSerialization ser = new GridHadoopSerializationWrapper(new WritableSerialization(), IntWritable.class);
-
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-        DataOutput out = new DataOutputStream(buf);
-
-        ser.write(out, new IntWritable(3));
-        ser.write(out, new IntWritable(-5));
-
-        assertEquals("[0, 0, 0, 3, -1, -1, -1, -5]", Arrays.toString(buf.toByteArray()));
-
-        DataInput in = new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
-
-        assertEquals(3, ((IntWritable)ser.read(in, null)).get());
-        assertEquals(-5, ((IntWritable)ser.read(in, null)).get());
-    }
-
-    /**
-     * Tests read/write of Integer via native JavaleSerialization.
-     * @throws Exception If fails.
-     */
-    public void testIntJavaSerialization() throws Exception {
-        GridHadoopSerialization ser = new GridHadoopSerializationWrapper(new JavaSerialization(), Integer.class);
-
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-        DataOutput out = new DataOutputStream(buf);
-
-        ser.write(out, 3);
-        ser.write(out, -5);
-        ser.close();
-
-        DataInput in = new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
-
-        assertEquals(3, ((Integer)ser.read(in, null)).intValue());
-        assertEquals(-5, ((Integer)ser.read(in, null)).intValue());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
deleted file mode 100644
index c086719..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSplitWrapperSelfTest.java
+++ /dev/null
@@ -1,68 +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.hadoop.fs.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.apache.ignite.testframework.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Self test of {@link GridHadoopSplitWrapper}.
- */
-public class GridHadoopSplitWrapperSelfTest extends GridHadoopAbstractSelfTest {
-    /**
-     * Tests serialization of wrapper and the wrapped native split.
-     * @throws Exception If fails.
-     */
-    public void testSerialization() throws Exception {
-        FileSplit nativeSplit = new FileSplit(new Path("/path/to/file"), 100, 500, new String[]{"host1", "host2"});
-
-        assertEquals("/path/to/file:100+500", nativeSplit.toString());
-
-        GridHadoopSplitWrapper split = HadoopUtils.wrapSplit(10, nativeSplit, nativeSplit.getLocations());
-
-        assertEquals("[host1, host2]", Arrays.toString(split.hosts()));
-
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-        ObjectOutput out = new ObjectOutputStream(buf);
-
-        out.writeObject(split);
-
-        ObjectInput in = new ObjectInputStream(new ByteArrayInputStream(buf.toByteArray()));
-
-        final GridHadoopSplitWrapper res = (GridHadoopSplitWrapper)in.readObject();
-
-        assertEquals("/path/to/file:100+500", HadoopUtils.unwrapSplit(res).toString());
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                res.hosts();
-
-                return null;
-            }
-        }, AssertionError.class, null);
-    }
-
-
-}


[05/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
new file mode 100644
index 0000000..6915d17
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -0,0 +1,552 @@
+/*
+ * 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.processors.hadoop.v2.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+import org.objectweb.asm.*;
+import org.objectweb.asm.commons.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Class loader allowing explicitly load classes without delegation to parent class loader.
+ * Also supports class parsing for finding dependencies which contain transitive dependencies
+ * unavailable for parent.
+ */
+public class HadoopClassLoader extends URLClassLoader {
+    /**
+     * We are very parallel capable.
+     */
+    static {
+        registerAsParallelCapable();
+    }
+
+    /** */
+    private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopClassLoader.class.getClassLoader();
+
+    /** */
+    private static final Collection<URL> appJars = F.asList(APP_CLS_LDR.getURLs());
+
+    /** */
+    private static volatile Collection<URL> hadoopJars;
+
+    /** */
+    private static final Map<String, Boolean> cache = new ConcurrentHashMap8<>();
+
+    /** */
+    private static final Map<String, byte[]> bytesCache = new ConcurrentHashMap8<>();
+
+    /**
+     * @param urls Urls.
+     */
+    public HadoopClassLoader(URL[] urls) {
+        super(addHadoopUrls(urls), APP_CLS_LDR);
+
+        assert !(getParent() instanceof HadoopClassLoader);
+    }
+
+    /**
+     * Need to parse only Ignite Hadoop and IGFS classes.
+     *
+     * @param cls Class name.
+     * @return {@code true} if we need to check this class.
+     */
+    private static boolean isIgfsHadoop(String cls) {
+        String ignitePackagePrefix = "org.apache.ignite";
+        int len = ignitePackagePrefix.length();
+
+        return cls.startsWith(ignitePackagePrefix) && (cls.indexOf("igfs.", len) != -1 || cls.indexOf(".fs.", len) != -1 || cls.indexOf("hadoop.", len) != -1);
+    }
+
+    /**
+     * @param cls Class name.
+     * @return {@code true} If this is Hadoop class.
+     */
+    private static boolean isHadoop(String cls) {
+        return cls.startsWith("org.apache.hadoop.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+        try {
+            if (isHadoop(name)) { // Always load Hadoop classes explicitly, since Hadoop can be available in App classpath.
+                if (name.endsWith(".util.ShutdownHookManager"))  // Dirty hack to get rid of Hadoop shutdown hooks.
+                    return loadFromBytes(name, GridHadoopShutdownHookManager.class.getName());
+                else if (name.endsWith(".util.NativeCodeLoader"))
+                    return loadFromBytes(name, GridHadoopNativeCodeLoader.class.getName());
+
+                return loadClassExplicitly(name, resolve);
+            }
+
+            if (isIgfsHadoop(name)) { // For Ignite Hadoop and IGFS classes we have to check if they depend on Hadoop.
+                Boolean hasDeps = cache.get(name);
+
+                if (hasDeps == null) {
+                    hasDeps = hasExternalDependencies(name, new HashSet<String>());
+
+                    cache.put(name, hasDeps);
+                }
+
+                if (hasDeps)
+                    return loadClassExplicitly(name, resolve);
+            }
+
+            return super.loadClass(name, resolve);
+        }
+        catch (NoClassDefFoundError | ClassNotFoundException e) {
+            throw new ClassNotFoundException("Failed to load class: " + name, e);
+        }
+    }
+
+    /**
+     * @param name Name.
+     * @param replace Replacement.
+     * @return Class.
+     */
+    private Class<?> loadFromBytes(final String name, final String replace) {
+        synchronized (getClassLoadingLock(name)) {
+            // First, check if the class has already been loaded
+            Class c = findLoadedClass(name);
+
+            if (c != null)
+                return c;
+
+            byte[] bytes = bytesCache.get(name);
+
+            if (bytes == null) {
+                InputStream in = loadClassBytes(getParent(), replace);
+
+                ClassReader rdr;
+
+                try {
+                    rdr = new ClassReader(in);
+                }
+                catch (IOException e) {
+                    throw new RuntimeException(e);
+                }
+
+                ClassWriter w = new ClassWriter(Opcodes.ASM4);
+
+                rdr.accept(new RemappingClassAdapter(w, new Remapper() {
+                    /** */
+                    String replaceType = replace.replace('.', '/');
+
+                    /** */
+                    String nameType = name.replace('.', '/');
+
+                    @Override public String map(String type) {
+                        if (type.equals(replaceType))
+                            return nameType;
+
+                        return type;
+                    }
+                }), ClassReader.EXPAND_FRAMES);
+
+                bytes = w.toByteArray();
+
+                bytesCache.put(name, bytes);
+            }
+
+            return defineClass(name, bytes, 0, bytes.length);
+        }
+    }
+
+    /**
+     * @param name Class name.
+     * @param resolve Resolve class.
+     * @return Class.
+     * @throws ClassNotFoundException If failed.
+     */
+    private Class<?> loadClassExplicitly(String name, boolean resolve) throws ClassNotFoundException {
+        synchronized (getClassLoadingLock(name)) {
+            // First, check if the class has already been loaded
+            Class c = findLoadedClass(name);
+
+            if (c == null) {
+                long t1 = System.nanoTime();
+
+                c = findClass(name);
+
+                // this is the defining class loader; record the stats
+                sun.misc.PerfCounter.getFindClassTime().addElapsedTimeFrom(t1);
+                sun.misc.PerfCounter.getFindClasses().increment();
+            }
+
+            if (resolve)
+                resolveClass(c);
+
+            return c;
+        }
+    }
+
+    /**
+     * @param ldr Loader.
+     * @param clsName Class.
+     * @return Input stream.
+     */
+    @Nullable private InputStream loadClassBytes(ClassLoader ldr, String clsName) {
+        return ldr.getResourceAsStream(clsName.replace('.', '/') + ".class");
+    }
+
+    /**
+     * @param clsName Class name.
+     * @return {@code true} If the class has external dependencies.
+     */
+    boolean hasExternalDependencies(final String clsName, final Set<String> visited) {
+        if (isHadoop(clsName)) // Hadoop must not be in classpath but Idea sucks, so filtering explicitly as external.
+            return true;
+
+        // Try to get from parent to check if the type accessible.
+        InputStream in = loadClassBytes(getParent(), clsName);
+
+        if (in == null) // The class is external itself, it must be loaded from this class loader.
+            return true;
+
+        if (!isIgfsHadoop(clsName)) // Other classes should not have external dependencies.
+            return false;
+
+        final ClassReader rdr;
+
+        try {
+            rdr = new ClassReader(in);
+        }
+        catch (IOException e) {
+            throw new RuntimeException("Failed to read class: " + clsName, e);
+        }
+
+        visited.add(clsName);
+
+        final AtomicBoolean hasDeps = new AtomicBoolean();
+
+        rdr.accept(new ClassVisitor(Opcodes.ASM4) {
+            AnnotationVisitor av = new AnnotationVisitor(Opcodes.ASM4) {
+                // TODO
+            };
+
+            FieldVisitor fv = new FieldVisitor(Opcodes.ASM4) {
+                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
+                    onType(desc);
+
+                    return av;
+                }
+            };
+
+            MethodVisitor mv = new MethodVisitor(Opcodes.ASM4) {
+                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
+                    onType(desc);
+
+                    return av;
+                }
+
+                @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
+                    onType(desc);
+
+                    return av;
+                }
+
+                @Override public AnnotationVisitor visitAnnotationDefault() {
+                    return av;
+                }
+
+                @Override public void visitFieldInsn(int i, String owner, String name, String desc) {
+                    onType(owner);
+                    onType(desc);
+                }
+
+                @Override public void visitFrame(int i, int i2, Object[] locTypes, int i3, Object[] stackTypes) {
+                    for (Object o : locTypes) {
+                        if (o instanceof String)
+                            onType((String)o);
+                    }
+
+                    for (Object o : stackTypes) {
+                        if (o instanceof String)
+                            onType((String)o);
+                    }
+                }
+
+                @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
+                    Label lb2, int i) {
+                    onType(desc);
+                }
+
+                @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
+                    onType(owner);
+                }
+
+                @Override public void visitMultiANewArrayInsn(String desc, int dim) {
+                    onType(desc);
+                }
+
+                @Override public void visitTryCatchBlock(Label lb, Label lb2, Label lb3, String e) {
+                    onType(e);
+                }
+            };
+
+            void onClass(String depCls) {
+                assert validateClassName(depCls) : depCls;
+
+                if (depCls.startsWith("java.")) // Filter out platform classes.
+                    return;
+
+                if (visited.contains(depCls))
+                    return;
+
+                Boolean res = cache.get(depCls);
+
+                if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
+                    hasDeps.set(true);
+            }
+
+            void onType(String type) {
+                if (type == null)
+                    return;
+
+                int off = 0;
+
+                while (type.charAt(off) == '[')
+                    off++; // Handle arrays.
+
+                if (off != 0)
+                    type = type.substring(off);
+
+                if (type.length() == 1)
+                    return; // Get rid of primitives.
+
+                if (type.charAt(type.length() - 1) == ';') {
+                    assert type.charAt(0) == 'L' : type;
+
+                    type = type.substring(1, type.length() - 1);
+                }
+
+                type = type.replace('/', '.');
+
+                onClass(type);
+            }
+
+            @Override public void visit(int i, int i2, String name, String signature, String superName,
+                String[] ifaces) {
+                onType(superName);
+
+                if (ifaces != null) {
+                    for (String iface : ifaces)
+                        onType(iface);
+                }
+            }
+
+            @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
+                onType(desc);
+
+                return av;
+            }
+
+            @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
+                onType(name);
+            }
+
+            @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
+                onType(desc);
+
+                return fv;
+            }
+
+            @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
+                String[] exceptions) {
+                if (exceptions != null) {
+                    for (String e : exceptions)
+                        onType(e);
+                }
+
+                return mv;
+            }
+        }, 0);
+
+        if (hasDeps.get()) // We already know that we have dependencies, no need to check parent.
+            return true;
+
+        // Here we are known to not have any dependencies but possibly we have a parent which have them.
+        int idx = clsName.lastIndexOf('$');
+
+        if (idx == -1) // No parent class.
+            return false;
+
+        String parentCls = clsName.substring(0, idx);
+
+        if (visited.contains(parentCls))
+            return false;
+
+        Boolean res = cache.get(parentCls);
+
+        if (res == null)
+            res = hasExternalDependencies(parentCls, visited);
+
+        return res;
+    }
+
+    /**
+     * @param name Class name.
+     * @return {@code true} If this is a valid class name.
+     */
+    private static boolean validateClassName(String name) {
+        int len = name.length();
+
+        if (len <= 1)
+            return false;
+
+        if (!Character.isJavaIdentifierStart(name.charAt(0)))
+            return false;
+
+        boolean hasDot = false;
+
+        for (int i = 1; i < len; i++) {
+            char c = name.charAt(i);
+
+            if (c == '.')
+                hasDot = true;
+            else if (!Character.isJavaIdentifierPart(c))
+                return false;
+        }
+
+        return hasDot;
+    }
+
+    /**
+     * @param name Variable name.
+     * @param dflt Default.
+     * @return Value.
+     */
+    private static String getEnv(String name, String dflt) {
+        String res = System.getProperty(name);
+
+        if (F.isEmpty(res))
+            res = System.getenv(name);
+
+        return F.isEmpty(res) ? dflt : res;
+    }
+
+    /**
+     * @param res Result.
+     * @param dir Directory.
+     * @param startsWith Starts with prefix.
+     * @throws MalformedURLException If failed.
+     */
+    private static void addUrls(Collection<URL> res, File dir, final String startsWith) throws Exception {
+        File[] files = dir.listFiles(new FilenameFilter() {
+            @Override public boolean accept(File dir, String name) {
+                return startsWith == null || name.startsWith(startsWith);
+            }
+        });
+
+        if (files == null)
+            throw new IOException("Path is not a directory: " + dir);
+
+        for (File file : files)
+            res.add(file.toURI().toURL());
+    }
+
+    /**
+     * @param urls URLs.
+     * @return URLs.
+     */
+    private static URL[] addHadoopUrls(URL[] urls) {
+        Collection<URL> hadoopJars;
+
+        try {
+            hadoopJars = hadoopUrls();
+        }
+        catch (IgniteCheckedException e) {
+            throw new RuntimeException(e);
+        }
+
+        ArrayList<URL> list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length));
+
+        list.addAll(appJars);
+        list.addAll(hadoopJars);
+
+        if (!F.isEmpty(urls))
+            list.addAll(F.asList(urls));
+
+        return list.toArray(new URL[list.size()]);
+    }
+
+    /**
+     * @return HADOOP_HOME Variable.
+     */
+    @Nullable public static String hadoopHome() {
+        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
+    }
+
+    /**
+     * @return Collection of jar URLs.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Collection<URL> hadoopUrls() throws IgniteCheckedException {
+        Collection<URL> hadoopUrls = hadoopJars;
+
+        if (hadoopUrls != null)
+            return hadoopUrls;
+
+        synchronized (HadoopClassLoader.class) {
+            hadoopUrls = hadoopJars;
+
+            if (hadoopUrls != null)
+                return hadoopUrls;
+
+            hadoopUrls = new ArrayList<>();
+
+            String hadoopPrefix = hadoopHome();
+
+            if (F.isEmpty(hadoopPrefix))
+                throw new IgniteCheckedException("Failed resolve Hadoop installation location. Either HADOOP_PREFIX or " +
+                    "HADOOP_HOME environment variables must be set.");
+
+            String commonHome = getEnv("HADOOP_COMMON_HOME", hadoopPrefix + "/share/hadoop/common");
+            String hdfsHome = getEnv("HADOOP_HDFS_HOME", hadoopPrefix + "/share/hadoop/hdfs");
+            String mapredHome = getEnv("HADOOP_MAPRED_HOME", hadoopPrefix + "/share/hadoop/mapreduce");
+
+            try {
+                addUrls(hadoopUrls, new File(commonHome + "/lib"), null);
+                addUrls(hadoopUrls, new File(hdfsHome + "/lib"), null);
+                addUrls(hadoopUrls, new File(mapredHome + "/lib"), null);
+
+                addUrls(hadoopUrls, new File(hdfsHome), "hadoop-hdfs-");
+
+                addUrls(hadoopUrls, new File(commonHome), "hadoop-common-");
+                addUrls(hadoopUrls, new File(commonHome), "hadoop-auth-");
+                addUrls(hadoopUrls, new File(commonHome + "/lib"), "hadoop-auth-");
+
+                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-common");
+                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-core");
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+
+            hadoopJars = hadoopUrls;
+
+            return hadoopUrls;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopComponent.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopComponent.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopComponent.java
new file mode 100644
index 0000000..cea11eb
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopComponent.java
@@ -0,0 +1,61 @@
+/*
+ * 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.*;
+
+/**
+ * Abstract class for all hadoop components.
+ */
+public abstract class HadoopComponent {
+    /** Hadoop context. */
+    protected HadoopContext ctx;
+
+    /** Logger. */
+    protected IgniteLogger log;
+
+    /**
+     * @param ctx Hadoop context.
+     */
+    public void start(HadoopContext ctx) throws IgniteCheckedException {
+        this.ctx = ctx;
+
+        log = ctx.kernalContext().log(getClass());
+    }
+
+    /**
+     * Stops manager.
+     */
+    public void stop(boolean cancel) {
+        // No-op.
+    }
+
+    /**
+     * Callback invoked when all grid components are started.
+     */
+    public void onKernalStart() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /**
+     * Callback invoked before all grid components are stopped.
+     */
+    public void onKernalStop(boolean cancel) {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
new file mode 100644
index 0000000..bb707c8
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
@@ -0,0 +1,196 @@
+/*
+ * 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.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+
+/**
+ * Hadoop accelerator context.
+ */
+public class HadoopContext {
+    /** Kernal context. */
+    private GridKernalContext ctx;
+
+    /** Hadoop configuration. */
+    private GridHadoopConfiguration cfg;
+
+    /** Job tracker. */
+    private HadoopJobTracker jobTracker;
+
+    /** External task executor. */
+    private HadoopTaskExecutorAdapter taskExecutor;
+
+    /** */
+    private HadoopShuffle shuffle;
+
+    /** Managers list. */
+    private List<HadoopComponent> components = new ArrayList<>();
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public HadoopContext(
+        GridKernalContext ctx,
+        GridHadoopConfiguration cfg,
+        HadoopJobTracker jobTracker,
+        HadoopTaskExecutorAdapter taskExecutor,
+        HadoopShuffle shuffle
+    ) {
+        this.ctx = ctx;
+        this.cfg = cfg;
+
+        this.jobTracker = add(jobTracker);
+        this.taskExecutor = add(taskExecutor);
+        this.shuffle = add(shuffle);
+    }
+
+    /**
+     * Gets list of managers.
+     *
+     * @return List of managers.
+     */
+    public List<HadoopComponent> components() {
+        return components;
+    }
+
+    /**
+     * Gets kernal context.
+     *
+     * @return Grid kernal context instance.
+     */
+    public GridKernalContext kernalContext() {
+        return ctx;
+    }
+
+    /**
+     * Gets Hadoop configuration.
+     *
+     * @return Hadoop configuration.
+     */
+    public GridHadoopConfiguration configuration() {
+        return cfg;
+    }
+
+    /**
+     * Gets local node ID. Shortcut for {@code kernalContext().localNodeId()}.
+     *
+     * @return Local node ID.
+     */
+    public UUID localNodeId() {
+        return ctx.localNodeId();
+    }
+
+    /**
+     * Gets local node order.
+     *
+     * @return Local node order.
+     */
+    public long localNodeOrder() {
+        assert ctx.discovery() != null;
+
+        return ctx.discovery().localNode().order();
+    }
+
+    /**
+     * @return Hadoop-enabled nodes.
+     */
+    public Collection<ClusterNode> nodes() {
+        return ctx.discovery().cacheNodes(CU.SYS_CACHE_HADOOP_MR, ctx.discovery().topologyVersion());
+    }
+
+    /**
+     * @return {@code True} if
+     */
+    public boolean jobUpdateLeader() {
+        long minOrder = Long.MAX_VALUE;
+        ClusterNode minOrderNode = null;
+
+        for (ClusterNode node : nodes()) {
+            if (node.order() < minOrder) {
+                minOrder = node.order();
+                minOrderNode = node;
+            }
+        }
+
+        assert minOrderNode != null;
+
+        return localNodeId().equals(minOrderNode.id());
+    }
+
+    /**
+     * @param meta Job metadata.
+     * @return {@code true} If local node is participating in job execution.
+     */
+    public boolean isParticipating(GridHadoopJobMetadata meta) {
+        UUID locNodeId = localNodeId();
+
+        if (locNodeId.equals(meta.submitNodeId()))
+            return true;
+
+        GridHadoopMapReducePlan plan = meta.mapReducePlan();
+
+        return plan.mapperNodeIds().contains(locNodeId) || plan.reducerNodeIds().contains(locNodeId) || jobUpdateLeader();
+    }
+
+    /**
+     * @return Jon tracker instance.
+     */
+    public HadoopJobTracker jobTracker() {
+        return jobTracker;
+    }
+
+    /**
+     * @return Task executor.
+     */
+    public HadoopTaskExecutorAdapter taskExecutor() {
+        return taskExecutor;
+    }
+
+    /**
+     * @return Shuffle.
+     */
+    public HadoopShuffle shuffle() {
+        return shuffle;
+    }
+
+    /**
+     * @return Map-reduce planner.
+     */
+    public GridHadoopMapReducePlanner planner() {
+        return cfg.getMapReducePlanner();
+    }
+
+    /**
+     * Adds component.
+     *
+     * @param c Component to add.
+     * @return Added manager.
+     */
+    private <C extends HadoopComponent> C add(C c) {
+        components.add(c);
+
+        return c;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounterGroup.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounterGroup.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounterGroup.java
index bdf8fc6..8655e14 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounterGroup.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounterGroup.java
@@ -24,7 +24,7 @@ import java.io.*;
 import java.util.*;
 
 /**
- * Hadoop counter group adapter.
+ * Hadoop +counter group adapter.
  */
 class HadoopCounterGroup implements CounterGroup {
     /** Counters. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
new file mode 100644
index 0000000..370b82d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -0,0 +1,163 @@
+/*
+ * 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.processors.hadoop.v2.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.lang.reflect.*;
+import java.util.*;
+
+/**
+ * Hadoop job info based on default Hadoop configuration.
+ */
+public class HadoopDefaultJobInfo implements GridHadoopJobInfo, Externalizable {
+    /** */
+    private static final long serialVersionUID = 5489900236464999951L;
+
+    /** {@code true} If job has combiner. */
+    private boolean hasCombiner;
+
+    /** Number of reducers configured for job. */
+    private int numReduces;
+
+    /** Configuration. */
+    private Map<String,String> props = new HashMap<>();
+
+    /** Job name. */
+    private String jobName;
+
+    /** User name. */
+    private String user;
+
+    /** */
+    private static volatile Class<?> jobCls;
+
+    /**
+     * Default constructor required by {@link Externalizable}.
+     */
+    public HadoopDefaultJobInfo() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param jobName Job name.
+     * @param user User name.
+     * @param hasCombiner {@code true} If job has combiner.
+     * @param numReduces Number of reducers configured for job.
+     * @param props All other properties of the job.
+     */
+    public HadoopDefaultJobInfo(String jobName, String user, boolean hasCombiner, int numReduces,
+        Map<String, String> props) {
+        this.jobName = jobName;
+        this.user = user;
+        this.hasCombiner = hasCombiner;
+        this.numReduces = numReduces;
+        this.props = props;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String property(String name) {
+        return props.get(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJob createJob(GridHadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        try {
+            Class<?> jobCls0 = jobCls;
+
+            if (jobCls0 == null) { // It is enough to have only one class loader with only Hadoop classes.
+                synchronized (HadoopDefaultJobInfo.class) {
+                    if ((jobCls0 = jobCls) == null) {
+                        HadoopClassLoader ldr = new HadoopClassLoader(null);
+
+                        jobCls = jobCls0 = ldr.loadClass(GridHadoopV2Job.class.getName());
+                    }
+                }
+            }
+
+            Constructor<?> constructor = jobCls0.getConstructor(GridHadoopJobId.class, HadoopDefaultJobInfo.class,
+                IgniteLogger.class);
+
+            return (GridHadoopJob)constructor.newInstance(jobId, this, log);
+        }
+        // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call.
+        catch (Throwable t) {
+            throw new IgniteCheckedException(t);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasCombiner() {
+        return hasCombiner;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasReducer() {
+        return reducers() > 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int reducers() {
+        return numReduces;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String jobName() {
+        return jobName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return user;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeString(out, jobName);
+        U.writeString(out, user);
+
+        out.writeBoolean(hasCombiner);
+        out.writeInt(numReduces);
+
+        U.writeStringMap(out, props);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobName = U.readString(in);
+        user = U.readString(in);
+
+        hasCombiner = in.readBoolean();
+        numReduces = in.readInt();
+
+        props = U.readStringMap(in);
+    }
+
+    /**
+     * @return Properties of the job.
+     */
+    public Map<String, String> properties() {
+        return props;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
new file mode 100644
index 0000000..80fd995
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
@@ -0,0 +1,132 @@
+/*
+ * 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.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Hadoop facade implementation.
+ */
+public class HadoopImpl implements GridHadoop {
+    /** Hadoop processor. */
+    private final IgniteHadoopProcessor proc;
+
+    /** Busy lock. */
+    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
+
+    /**
+     * Constructor.
+     *
+     * @param proc Hadoop processor.
+     */
+    HadoopImpl(IgniteHadoopProcessor proc) {
+        this.proc = proc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopConfiguration configuration() {
+        return proc.config();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobId nextJobId() {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.nextJobId();
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get next job ID (grid is stopping).");
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.submit(jobId, jobInfo);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to submit job (grid is stopping).");
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.status(jobId);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get job status (grid is stopping).");
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.counters(jobId);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get job counters (grid is stopping).");
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.finishFuture(jobId);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to get job finish future (grid is stopping).");
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                return proc.kill(jobId);
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to kill job (grid is stopping).");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSetup.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSetup.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSetup.java
new file mode 100644
index 0000000..35df5da
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSetup.java
@@ -0,0 +1,505 @@
+/*
+ * 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.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.net.*;
+import java.nio.file.*;
+import java.text.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.IgniteVersionUtils.*;
+
+/**
+ * Setup tool to configure Hadoop client.
+ */
+public class HadoopSetup {
+    /** */
+    public static final String WINUTILS_EXE = "winutils.exe";
+
+    /** */
+    private static final FilenameFilter IGNITE_JARS = new FilenameFilter() {
+        @Override public boolean accept(File dir, String name) {
+            return name.startsWith("ignite-") && name.endsWith(".jar");
+        }
+    };
+
+    /**
+     * The main method.
+     * @param ignore Params.
+     */
+    public static void main(String[] ignore) {
+        X.println(
+            "   __________  ________________ ",
+            "  /  _/ ___/ |/ /  _/_  __/ __/ ",
+            " _/ // (_ /    // /  / / / _/   ",
+            "/___/\\___/_/|_/___/ /_/ /___/  ",
+            "                for Apache Hadoop        ",
+            "  ");
+
+        println("Version " + ACK_VER_STR);
+
+        configureHadoop();
+    }
+
+    /**
+     * This operation prepares the clean unpacked Hadoop distributive to work as client with Ignite-Hadoop.
+     * It performs these operations:
+     * <ul>
+     *     <li>Check for setting of HADOOP_HOME environment variable.</li>
+     *     <li>Try to resolve HADOOP_COMMON_HOME or evaluate it relative to HADOOP_HOME.</li>
+     *     <li>In Windows check if winutils.exe exists and try to fix issue with some restrictions.</li>
+     *     <li>In Windows check new line character issues in CMD scripts.</li>
+     *     <li>Scan Hadoop lib directory to detect Ignite JARs. If these don't exist tries to create ones.</li>
+     * </ul>
+     */
+    private static void configureHadoop() {
+        String igniteHome = U.getIgniteHome();
+
+        println("IGNITE_HOME is set to '" + igniteHome + "'.");
+
+        checkIgniteHome(igniteHome);
+
+        String homeVar = "HADOOP_HOME";
+        String hadoopHome = System.getenv(homeVar);
+
+        if (F.isEmpty(hadoopHome)) {
+            homeVar = "HADOOP_PREFIX";
+            hadoopHome = System.getenv(homeVar);
+        }
+
+        if (F.isEmpty(hadoopHome))
+            exit("Neither HADOOP_HOME nor HADOOP_PREFIX environment variable is set. Please set one of them to a " +
+                "valid Hadoop installation directory and run setup tool again.", null);
+
+        hadoopHome = hadoopHome.replaceAll("\"", "");
+
+        println(homeVar + " is set to '" + hadoopHome + "'.");
+
+        String hiveHome = System.getenv("HIVE_HOME");
+
+        if (!F.isEmpty(hiveHome)) {
+            hiveHome = hiveHome.replaceAll("\"", "");
+
+            println("HIVE_HOME is set to '" + hiveHome + "'.");
+        }
+
+        File hadoopDir = new File(hadoopHome);
+
+        if (!hadoopDir.exists())
+            exit("Hadoop installation folder does not exist.", null);
+
+        if (!hadoopDir.isDirectory())
+            exit("HADOOP_HOME must point to a directory.", null);
+
+        if (!hadoopDir.canRead())
+            exit("Hadoop installation folder can not be read. Please check permissions.", null);
+
+        File hadoopCommonDir;
+
+        String hadoopCommonHome = System.getenv("HADOOP_COMMON_HOME");
+
+        if (F.isEmpty(hadoopCommonHome)) {
+            hadoopCommonDir = new File(hadoopDir, "share/hadoop/common");
+
+            println("HADOOP_COMMON_HOME is not set, will use '" + hadoopCommonDir.getPath() + "'.");
+        }
+        else {
+            println("HADOOP_COMMON_HOME is set to '" + hadoopCommonHome + "'.");
+
+            hadoopCommonDir = new File(hadoopCommonHome);
+        }
+
+        if (!hadoopCommonDir.canRead())
+            exit("Failed to read Hadoop common dir in '" + hadoopCommonHome + "'.", null);
+
+        File hadoopCommonLibDir = new File(hadoopCommonDir, "lib");
+
+        if (!hadoopCommonLibDir.canRead())
+            exit("Failed to read Hadoop 'lib' folder in '" + hadoopCommonLibDir.getPath() + "'.", null);
+
+        if (U.isWindows()) {
+            checkJavaPathSpaces();
+
+            File hadoopBinDir = new File(hadoopDir, "bin");
+
+            if (!hadoopBinDir.canRead())
+                exit("Failed to read subdirectory 'bin' in HADOOP_HOME.", null);
+
+            File winutilsFile = new File(hadoopBinDir, WINUTILS_EXE);
+
+            if (!winutilsFile.exists()) {
+                if (ask("File '" + WINUTILS_EXE + "' does not exist. " +
+                    "It may be replaced by a stub. Create it?")) {
+                    println("Creating file stub '" + winutilsFile.getAbsolutePath() + "'.");
+
+                    boolean ok = false;
+
+                    try {
+                        ok = winutilsFile.createNewFile();
+                    }
+                    catch (IOException ignore) {
+                        // No-op.
+                    }
+
+                    if (!ok)
+                        exit("Failed to create '" + WINUTILS_EXE + "' file. Please check permissions.", null);
+                }
+                else
+                    println("Ok. But Hadoop client probably will not work on Windows this way...");
+            }
+
+            processCmdFiles(hadoopDir, "bin", "sbin", "libexec");
+        }
+
+        File igniteLibs = new File(new File(igniteHome), "libs");
+
+        if (!igniteLibs.exists())
+            exit("Ignite 'libs' folder is not found.", null);
+
+        Collection<File> jarFiles = new ArrayList<>();
+
+        addJarsInFolder(jarFiles, igniteLibs);
+        addJarsInFolder(jarFiles, new File(igniteLibs, "ignite-hadoop"));
+
+        boolean jarsLinksCorrect = true;
+
+        for (File file : jarFiles) {
+            File link = new File(hadoopCommonLibDir, file.getName());
+
+            jarsLinksCorrect &= isJarLinkCorrect(link, file);
+
+            if (!jarsLinksCorrect)
+                break;
+        }
+
+        if (!jarsLinksCorrect) {
+            if (ask("Ignite JAR files are not found in Hadoop 'lib' directory. " +
+                "Create appropriate symbolic links?")) {
+                File[] oldIgniteJarFiles = hadoopCommonLibDir.listFiles(IGNITE_JARS);
+
+                if (oldIgniteJarFiles.length > 0 && ask("The Hadoop 'lib' directory contains JARs from other Ignite " +
+                    "installation. They must be deleted to continue. Continue?")) {
+                    for (File file : oldIgniteJarFiles) {
+                        println("Deleting file '" + file.getAbsolutePath() + "'.");
+
+                        if (!file.delete())
+                            exit("Failed to delete file '" + file.getPath() + "'.", null);
+                    }
+                }
+
+                for (File file : jarFiles) {
+                    File targetFile = new File(hadoopCommonLibDir, file.getName());
+
+                    try {
+                        println("Creating symbolic link '" + targetFile.getAbsolutePath() + "'.");
+
+                        Files.createSymbolicLink(targetFile.toPath(), file.toPath());
+                    }
+                    catch (IOException e) {
+                        if (U.isWindows()) {
+                            warn("Ability to create symbolic links is required!");
+                            warn("On Windows platform you have to grant permission 'Create symbolic links'");
+                            warn("to your user or run the Accelerator as Administrator.");
+                        }
+
+                        exit("Creating symbolic link failed! Check permissions.", e);
+                    }
+                }
+            }
+            else
+                println("Ok. But Hadoop client will not be able to talk to Ignite cluster without those JARs in classpath...");
+        }
+
+        File hadoopEtc = new File(hadoopDir, "etc" + File.separator + "hadoop");
+
+        File igniteDocs = new File(igniteHome, "docs");
+
+        if (!igniteDocs.canRead())
+            exit("Failed to read Ignite 'docs' folder at '" + igniteDocs.getAbsolutePath() + "'.", null);
+
+        if (hadoopEtc.canWrite()) { // TODO Bigtop
+            if (ask("Replace 'core-site.xml' and 'mapred-site.xml' files with preconfigured templates " +
+                "(existing files will be backed up)?")) {
+                replaceWithBackup(new File(igniteDocs, "core-site.ignite.xml"), new File(hadoopEtc, "core-site.xml"));
+
+                replaceWithBackup(new File(igniteDocs, "mapred-site.ignite.xml"), new File(hadoopEtc, "mapred-site.xml"));
+            }
+            else
+                println("Ok. You can configure them later, the templates are available at Ignite's 'docs' directory...");
+        }
+
+        if (!F.isEmpty(hiveHome)) {
+            File hiveConfDir = new File(hiveHome + File.separator + "conf");
+
+            if (!hiveConfDir.canWrite())
+                warn("Can not write to '" + hiveConfDir.getAbsolutePath() + "'. To run Hive queries you have to " +
+                    "configure 'hive-site.xml' manually. The template is available at Ignite's 'docs' directory.");
+            else if (ask("Replace 'hive-site.xml' with preconfigured template (existing file will be backed up)?"))
+                replaceWithBackup(new File(igniteDocs, "hive-site.ignite.xml"), new File(hiveConfDir, "hive-site.xml"));
+            else
+                println("Ok. You can configure it later, the template is available at Ignite's 'docs' directory...");
+        }
+
+        println("Apache Hadoop setup is complete.");
+    }
+
+    /**
+     * @param jarFiles Jars.
+     * @param folder Folder.
+     */
+    private static void addJarsInFolder(Collection<File> jarFiles, File folder) {
+        if (!folder.exists())
+            exit("Folder '" + folder.getAbsolutePath() + "' is not found.", null);
+
+        jarFiles.addAll(Arrays.asList(folder.listFiles(IGNITE_JARS)));
+    }
+
+    /**
+     * Checks that JAVA_HOME does not contain space characters.
+     */
+    private static void checkJavaPathSpaces() {
+        String javaHome = System.getProperty("java.home");
+
+        if (javaHome.contains(" ")) {
+            warn("Java installation path contains space characters!");
+            warn("Hadoop client will not be able to start using '" + javaHome + "'.");
+            warn("Please install JRE to path which does not contain spaces and point JAVA_HOME to that installation.");
+        }
+    }
+
+    /**
+     * Checks Ignite home.
+     *
+     * @param igniteHome Ignite home.
+     */
+    private static void checkIgniteHome(String igniteHome) {
+        URL jarUrl = U.class.getProtectionDomain().getCodeSource().getLocation();
+
+        try {
+            Path jar = Paths.get(jarUrl.toURI());
+            Path igHome = Paths.get(igniteHome);
+
+            if (!jar.startsWith(igHome))
+                exit("Ignite JAR files are not under IGNITE_HOME.", null);
+        }
+        catch (Exception e) {
+            exit(e.getMessage(), e);
+        }
+    }
+
+    /**
+     * Replaces target file with source file.
+     *
+     * @param from From.
+     * @param to To.
+     */
+    private static void replaceWithBackup(File from, File to) {
+        if (!from.canRead())
+            exit("Failed to read source file '" + from.getAbsolutePath() + "'.", null);
+
+        println("Replacing file '" + to.getAbsolutePath() + "'.");
+
+        try {
+            U.copy(from, renameToBak(to), true);
+        }
+        catch (IOException e) {
+            exit("Failed to replace file '" + to.getAbsolutePath() + "'.", e);
+        }
+    }
+
+    /**
+     * Renames file for backup.
+     *
+     * @param file File.
+     * @return File.
+     */
+    private static File renameToBak(File file) {
+        DateFormat fmt = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss");
+
+        if (file.exists() && !file.renameTo(new File(file.getAbsolutePath() + "." + fmt.format(new Date()) + ".bak")))
+            exit("Failed to rename file '" + file.getPath() + "'.", null);
+
+        return file;
+    }
+
+    /**
+     * Checks if link is correct.
+     *
+     * @param link Symbolic link.
+     * @param correctTarget Correct link target.
+     * @return {@code true} If link target is correct.
+     */
+    private static boolean isJarLinkCorrect(File link, File correctTarget) {
+        if (!Files.isSymbolicLink(link.toPath()))
+            return false; // It is a real file or it does not exist.
+
+        Path target = null;
+
+        try {
+            target = Files.readSymbolicLink(link.toPath());
+        }
+        catch (IOException e) {
+            exit("Failed to read symbolic link: " + link.getAbsolutePath(), e);
+        }
+
+        return Files.exists(target) && target.toFile().equals(correctTarget);
+    }
+
+    /**
+     * Writes the question end read the boolean answer from the console.
+     *
+     * @param question Question to write.
+     * @return {@code true} if user inputs 'Y' or 'y', {@code false} otherwise.
+     */
+    private static boolean ask(String question) {
+        X.println();
+        X.print(" <  " + question + " (Y/N): ");
+
+        String answer = null;
+
+        if (!F.isEmpty(System.getenv("IGNITE_HADOOP_SETUP_YES")))
+            answer = "Y";
+        else {
+            BufferedReader br = new BufferedReader(new InputStreamReader(System.in));
+
+            try {
+                answer = br.readLine();
+            }
+            catch (IOException e) {
+                exit("Failed to read answer: " + e.getMessage(), e);
+            }
+        }
+
+        if (answer != null && "Y".equals(answer.toUpperCase().trim())) {
+            X.println(" >  Yes.");
+
+            return true;
+        }
+        else {
+            X.println(" >  No.");
+
+            return false;
+        }
+    }
+
+    /**
+     * Exit with message.
+     *
+     * @param msg Exit message.
+     */
+    private static void exit(String msg, Exception e) {
+        X.println("    ");
+        X.println("  # " + msg);
+        X.println("  # Setup failed, exiting... ");
+
+        if (e != null && !F.isEmpty(System.getenv("IGNITE_HADOOP_SETUP_DEBUG")))
+            e.printStackTrace();
+
+        System.exit(1);
+    }
+
+    /**
+     * Prints message.
+     *
+     * @param msg Message.
+     */
+    private static void println(String msg) {
+        X.println("  > " + msg);
+    }
+
+    /**
+     * Prints warning.
+     *
+     * @param msg Message.
+     */
+    private static void warn(String msg) {
+        X.println("  ! " + msg);
+    }
+
+    /**
+     * Checks that CMD files have valid MS Windows new line characters. If not, writes question to console and reads the
+     * answer. If it's 'Y' then backups original files and corrects invalid new line characters.
+     *
+     * @param rootDir Root directory to process.
+     * @param dirs Directories inside of the root to process.
+     */
+    private static void processCmdFiles(File rootDir, String... dirs) {
+        boolean answer = false;
+
+        for (String dir : dirs) {
+            File subDir = new File(rootDir, dir);
+
+            File[] cmdFiles = subDir.listFiles(new FilenameFilter() {
+                @Override public boolean accept(File dir, String name) {
+                    return name.toLowerCase().endsWith(".cmd");
+                }
+            });
+
+            for (File file : cmdFiles) {
+                String content = null;
+
+                try (Scanner scanner = new Scanner(file)) {
+                    content = scanner.useDelimiter("\\Z").next();
+                }
+                catch (FileNotFoundException e) {
+                    exit("Failed to read file '" + file + "'.", e);
+                }
+
+                boolean invalid = false;
+
+                for (int i = 0; i < content.length(); i++) {
+                    if (content.charAt(i) == '\n' && (i == 0 || content.charAt(i - 1) != '\r')) {
+                        invalid = true;
+
+                        break;
+                    }
+                }
+
+                if (invalid) {
+                    answer = answer || ask("One or more *.CMD files has invalid new line character. Replace them?");
+
+                    if (!answer) {
+                        println("Ok. But Windows most probably will fail to execute them...");
+
+                        return;
+                    }
+
+                    println("Fixing newline characters in file '" + file.getAbsolutePath() + "'.");
+
+                    renameToBak(file);
+
+                    try (BufferedWriter writer = new BufferedWriter(new FileWriter(file))) {
+                        for (int i = 0; i < content.length(); i++) {
+                            if (content.charAt(i) == '\n' && (i == 0 || content.charAt(i - 1) != '\r'))
+                                writer.write("\r");
+
+                            writer.write(content.charAt(i));
+                        }
+                    }
+                    catch (IOException e) {
+                        exit("Failed to write file '" + file.getPath() + "': " + e.getMessage(), e);
+                    }
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskCancelledException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskCancelledException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskCancelledException.java
new file mode 100644
index 0000000..bb3d1cc
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskCancelledException.java
@@ -0,0 +1,35 @@
+/*
+ * 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.*;
+
+/**
+ * Exception that throws when the task is cancelling.
+ */
+public class HadoopTaskCancelledException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param msg Exception message.
+     */
+    public HadoopTaskCancelledException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
new file mode 100644
index 0000000..46594ce
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
@@ -0,0 +1,308 @@
+/*
+ * 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.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.JobPriority;
+import org.apache.hadoop.mapreduce.JobStatus;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Hadoop utility methods.
+ */
+public class HadoopUtils {
+    /** Property to store timestamp of new job id request. */
+    public static final String REQ_NEW_JOBID_TS_PROPERTY = "ignite.job.requestNewIdTs";
+
+    /** Property to store timestamp of response of new job id request. */
+    public static final String RESPONSE_NEW_JOBID_TS_PROPERTY = "ignite.job.responseNewIdTs";
+
+    /** Property to store timestamp of job submission. */
+    public static final String JOB_SUBMISSION_START_TS_PROPERTY = "ignite.job.submissionStartTs";
+
+    /** Property to set custom writer of job statistics. */
+    public static final String JOB_COUNTER_WRITER_PROPERTY = "ignite.counters.writer";
+
+    /** Staging constant. */
+    private static final String STAGING_CONSTANT = ".staging";
+
+    /** Old mapper class attribute. */
+    private static final String OLD_MAP_CLASS_ATTR = "mapred.mapper.class";
+
+    /** Old reducer class attribute. */
+    private static final String OLD_REDUCE_CLASS_ATTR = "mapred.reducer.class";
+
+    /**
+     * Wraps native split.
+     *
+     * @param id Split ID.
+     * @param split Split.
+     * @param hosts Hosts.
+     * @throws IOException If failed.
+     */
+    public static GridHadoopSplitWrapper wrapSplit(int id, Object split, String[] hosts) throws IOException {
+        ByteArrayOutputStream arr = new ByteArrayOutputStream();
+        ObjectOutput out = new ObjectOutputStream(arr);
+
+        assert split instanceof Writable;
+
+        ((Writable)split).write(out);
+
+        out.flush();
+
+        return new GridHadoopSplitWrapper(id, split.getClass().getName(), arr.toByteArray(), hosts);
+    }
+
+    /**
+     * Unwraps native split.
+     *
+     * @param o Wrapper.
+     * @return Split.
+     */
+    public static Object unwrapSplit(GridHadoopSplitWrapper o) {
+        try {
+            Writable w = (Writable)HadoopUtils.class.getClassLoader().loadClass(o.className()).newInstance();
+
+            w.readFields(new ObjectInputStream(new ByteArrayInputStream(o.bytes())));
+
+            return w;
+        }
+        catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    /**
+     * Convert Ignite job status to Hadoop job status.
+     *
+     * @param status Ignite job status.
+     * @return Hadoop job status.
+     */
+    public static JobStatus status(GridHadoopJobStatus status, Configuration conf) {
+        JobID jobId = new JobID(status.jobId().globalId().toString(), status.jobId().localId());
+
+        float setupProgress = 0;
+        float mapProgress = 0;
+        float reduceProgress = 0;
+        float cleanupProgress = 0;
+
+        JobStatus.State state = JobStatus.State.RUNNING;
+
+        switch (status.jobPhase()) {
+            case PHASE_SETUP:
+                setupProgress = 0.42f;
+
+                break;
+
+            case PHASE_MAP:
+                setupProgress = 1;
+                mapProgress = 1f - status.pendingMapperCnt() / (float)status.totalMapperCnt();
+
+                break;
+
+            case PHASE_REDUCE:
+                assert status.totalReducerCnt() > 0;
+
+                setupProgress = 1;
+                mapProgress = 1;
+                reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
+
+                break;
+
+            case PHASE_CANCELLING:
+            case PHASE_COMPLETE:
+                if (!status.isFailed()) {
+                    setupProgress = 1;
+                    mapProgress = 1;
+                    reduceProgress = 1;
+                    cleanupProgress = 1;
+
+                    state = JobStatus.State.SUCCEEDED;
+                }
+                else
+                    state = JobStatus.State.FAILED;
+
+                break;
+
+            default:
+                assert false;
+        }
+
+        return new JobStatus(jobId, setupProgress, mapProgress, reduceProgress, cleanupProgress, state,
+            JobPriority.NORMAL, status.user(), status.jobName(), jobFile(conf, status.user(), jobId).toString(), "N/A");
+    }
+
+    /**
+     * Gets staging area directory.
+     *
+     * @param conf Configuration.
+     * @param usr User.
+     * @return Staging area directory.
+     */
+    public static Path stagingAreaDir(Configuration conf, String usr) {
+        return new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR, MRJobConfig.DEFAULT_MR_AM_STAGING_DIR)
+            + Path.SEPARATOR + usr + Path.SEPARATOR + STAGING_CONSTANT);
+    }
+
+    /**
+     * Gets job file.
+     *
+     * @param conf Configuration.
+     * @param usr User.
+     * @param jobId Job ID.
+     * @return Job file.
+     */
+    public static Path jobFile(Configuration conf, String usr, JobID jobId) {
+        return new Path(stagingAreaDir(conf, usr), jobId.toString() + Path.SEPARATOR + MRJobConfig.JOB_CONF_FILE);
+    }
+
+    /**
+     * Checks the attribute in configuration is not set.
+     *
+     * @param attr Attribute name.
+     * @param msg Message for creation of exception.
+     * @throws IgniteCheckedException If attribute is set.
+     */
+    public static void ensureNotSet(Configuration cfg, String attr, String msg) throws IgniteCheckedException {
+        if (cfg.get(attr) != null)
+            throw new IgniteCheckedException(attr + " is incompatible with " + msg + " mode.");
+    }
+
+    /**
+     * Creates JobInfo from hadoop configuration.
+     *
+     * @param cfg Hadoop configuration.
+     * @return Job info.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static HadoopDefaultJobInfo createJobInfo(Configuration cfg) throws IgniteCheckedException {
+        JobConf jobConf = new JobConf(cfg);
+
+        boolean hasCombiner = jobConf.get("mapred.combiner.class") != null
+                || jobConf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null;
+
+        int numReduces = jobConf.getNumReduceTasks();
+
+        jobConf.setBooleanIfUnset("mapred.mapper.new-api", jobConf.get(OLD_MAP_CLASS_ATTR) == null);
+
+        if (jobConf.getUseNewMapper()) {
+            String mode = "new map API";
+
+            ensureNotSet(jobConf, "mapred.input.format.class", mode);
+            ensureNotSet(jobConf, OLD_MAP_CLASS_ATTR, mode);
+
+            if (numReduces != 0)
+                ensureNotSet(jobConf, "mapred.partitioner.class", mode);
+            else
+                ensureNotSet(jobConf, "mapred.output.format.class", mode);
+        }
+        else {
+            String mode = "map compatibility";
+
+            ensureNotSet(jobConf, MRJobConfig.INPUT_FORMAT_CLASS_ATTR, mode);
+            ensureNotSet(jobConf, MRJobConfig.MAP_CLASS_ATTR, mode);
+
+            if (numReduces != 0)
+                ensureNotSet(jobConf, MRJobConfig.PARTITIONER_CLASS_ATTR, mode);
+            else
+                ensureNotSet(jobConf, MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, mode);
+        }
+
+        if (numReduces != 0) {
+            jobConf.setBooleanIfUnset("mapred.reducer.new-api", jobConf.get(OLD_REDUCE_CLASS_ATTR) == null);
+
+            if (jobConf.getUseNewReducer()) {
+                String mode = "new reduce API";
+
+                ensureNotSet(jobConf, "mapred.output.format.class", mode);
+                ensureNotSet(jobConf, OLD_REDUCE_CLASS_ATTR, mode);
+            }
+            else {
+                String mode = "reduce compatibility";
+
+                ensureNotSet(jobConf, MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, mode);
+                ensureNotSet(jobConf, MRJobConfig.REDUCE_CLASS_ATTR, mode);
+            }
+        }
+
+        Map<String, String> props = new HashMap<>();
+
+        for (Map.Entry<String, String> entry : jobConf)
+            props.put(entry.getKey(), entry.getValue());
+
+        return new HadoopDefaultJobInfo(jobConf.getJobName(), jobConf.getUser(), hasCombiner, numReduces, props);
+    }
+
+    /**
+     * Throws new {@link IgniteCheckedException} with original exception is serialized into string.
+     * This is needed to transfer error outside the current class loader.
+     *
+     * @param e Original exception.
+     * @return IgniteCheckedException New exception.
+     */
+    public static IgniteCheckedException transformException(Throwable e) {
+        ByteArrayOutputStream os = new ByteArrayOutputStream();
+
+        e.printStackTrace(new PrintStream(os, true));
+
+        return new IgniteCheckedException(os.toString());
+    }
+
+    /**
+     * Returns work directory for job execution.
+     *
+     * @param locNodeId Local node ID.
+     * @param jobId Job ID.
+     * @return Working directory for job.
+     * @throws IgniteCheckedException If Failed.
+     */
+    public static File jobLocalDir(UUID locNodeId, GridHadoopJobId jobId) throws IgniteCheckedException {
+        return new File(new File(U.resolveWorkDirectory("hadoop", false), "node-" + locNodeId), "job_" + jobId);
+    }
+
+    /**
+     * Returns subdirectory of job working directory for task execution.
+     *
+     * @param locNodeId Local node ID.
+     * @param info Task info.
+     * @return Working directory for task.
+     * @throws IgniteCheckedException If Failed.
+     */
+    public static File taskLocalDir(UUID locNodeId, GridHadoopTaskInfo info) throws IgniteCheckedException {
+        File jobLocDir = jobLocalDir(locNodeId, info.jobId());
+
+        return new File(jobLocDir, info.type() + "_" + info.taskNumber() + "_" + info.attempt());
+    }
+
+    /**
+     * Constructor.
+     */
+    private HadoopUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
index 4ef9e35..63e4854 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
@@ -30,7 +30,7 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import java.util.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopClassLoader.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopClassLoader.*;
 
 /**
  * Hadoop processor.
@@ -41,7 +41,7 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
 
     /** Hadoop context. */
     @GridToStringExclude
-    private GridHadoopContext hctx;
+    private HadoopContext hctx;
 
     /** Hadoop facade for public API. */
     @GridToStringExclude
@@ -85,18 +85,18 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
         }
 
         if (ok) {
-            hctx = new GridHadoopContext(
+            hctx = new HadoopContext(
                 ctx,
                 cfg,
-                new GridHadoopJobTracker(),
-                cfg.isExternalExecution() ? new GridHadoopExternalTaskExecutor() : new GridHadoopEmbeddedTaskExecutor(),
-                new GridHadoopShuffle());
+                new HadoopJobTracker(),
+                cfg.isExternalExecution() ? new HadoopExternalTaskExecutor() : new HadoopEmbeddedTaskExecutor(),
+                new HadoopShuffle());
 
 
-            for (GridHadoopComponent c : hctx.components())
+            for (HadoopComponent c : hctx.components())
                 c.start(hctx);
 
-            hadoop = new GridHadoopImpl(this);
+            hadoop = new HadoopImpl(this);
         }
     }
 
@@ -112,10 +112,10 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
         if (hctx == null)
             return;
 
-        List<GridHadoopComponent> components = hctx.components();
+        List<HadoopComponent> components = hctx.components();
 
-        for (ListIterator<GridHadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
-            GridHadoopComponent c = it.previous();
+        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
+            HadoopComponent c = it.previous();
 
             c.stop(cancel);
         }
@@ -128,7 +128,7 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
         if (hctx == null)
             return;
 
-        for (GridHadoopComponent c : hctx.components())
+        for (HadoopComponent c : hctx.components())
             c.onKernalStart();
     }
 
@@ -139,10 +139,10 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
         if (hctx == null)
             return;
 
-        List<GridHadoopComponent> components = hctx.components();
+        List<HadoopComponent> components = hctx.components();
 
-        for (ListIterator<GridHadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
-            GridHadoopComponent c = it.previous();
+        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
+            HadoopComponent c = it.previous();
 
             c.onKernalStop(cancel);
         }
@@ -153,7 +153,7 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
      *
      * @return Hadoop context.
      */
-    public GridHadoopContext context() {
+    public HadoopContext context() {
         return hctx;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
index 55dcc4c..d603d76 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
@@ -52,7 +52,7 @@ public class GridHadoopFSCounterWriter implements GridHadoopCounterWriter {
 
         Configuration hadoopCfg = new Configuration();
 
-        for (Map.Entry<String, String> e : ((GridHadoopDefaultJobInfo)jobInfo).properties().entrySet())
+        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
             hadoopCfg.set(e.getKey(), e.getValue());
 
         String user = jobInfo.user();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
index d5ceebf..263a075 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
@@ -25,7 +25,7 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Counter for the job statistics accumulation.


[16/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksAllVersionsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksAllVersionsTest.java
index af3f872..a959472 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksAllVersionsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksAllVersionsTest.java
@@ -43,7 +43,7 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    public abstract GridHadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception;
+    public abstract HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception;
 
     /**
      * @return prefix of reducer output file name. It's "part-" for v1 and "part-r-" for v2 API
@@ -79,11 +79,11 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
         GridHadoopFileBlock fileBlock2 = new GridHadoopFileBlock(HOSTS, inFileUri, fileBlock1.length(),
                 igfs.info(inFile).length() - fileBlock1.length());
 
-        GridHadoopV2Job gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
 
         GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock1);
 
-        GridHadoopTestTaskContext ctx = new GridHadoopTestTaskContext(taskInfo, gridJob);
+        HadoopTestTaskContext ctx = new HadoopTestTaskContext(taskInfo, gridJob);
 
         ctx.mockOutput().clear();
 
@@ -110,11 +110,11 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
      * @return Context with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private GridHadoopTestTaskContext runTaskWithInput(GridHadoopV2Job gridJob, GridHadoopTaskType taskType,
+    private HadoopTestTaskContext runTaskWithInput(HadoopV2Job gridJob, GridHadoopTaskType taskType,
         int taskNum, String... words) throws IgniteCheckedException {
         GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(taskType, gridJob.id(), taskNum, 0, null);
 
-        GridHadoopTestTaskContext ctx = new GridHadoopTestTaskContext(taskInfo, gridJob);
+        HadoopTestTaskContext ctx = new HadoopTestTaskContext(taskInfo, gridJob);
 
         for (int i = 0; i < words.length; i+=2) {
             List<IntWritable> valList = new ArrayList<>();
@@ -136,7 +136,7 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
      * @throws Exception If fails.
      */
     public void testReduceTask() throws Exception {
-        GridHadoopV2Job gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
+        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
 
         runTaskWithInput(gridJob, GridHadoopTaskType.REDUCE, 0, "word1", "5", "word2", "10");
         runTaskWithInput(gridJob, GridHadoopTaskType.REDUCE, 1, "word3", "7", "word4", "15");
@@ -162,9 +162,9 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
      * @throws Exception If fails.
      */
     public void testCombinerTask() throws Exception {
-        GridHadoopV2Job gridJob = getHadoopJob("/", "/");
+        HadoopV2Job gridJob = getHadoopJob("/", "/");
 
-        GridHadoopTestTaskContext ctx =
+        HadoopTestTaskContext ctx =
             runTaskWithInput(gridJob, GridHadoopTaskType.COMBINE, 0, "word1", "5", "word2", "10");
 
         assertEquals("word1,5; word2,10", Joiner.on("; ").join(ctx.mockOutput()));
@@ -182,18 +182,18 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
      * @return Context of combine task with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private GridHadoopTestTaskContext runMapCombineTask(GridHadoopFileBlock fileBlock, GridHadoopV2Job gridJob)
+    private HadoopTestTaskContext runMapCombineTask(GridHadoopFileBlock fileBlock, HadoopV2Job gridJob)
         throws IgniteCheckedException {
         GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock);
 
-        GridHadoopTestTaskContext mapCtx = new GridHadoopTestTaskContext(taskInfo, gridJob);
+        HadoopTestTaskContext mapCtx = new HadoopTestTaskContext(taskInfo, gridJob);
 
         mapCtx.run();
 
         //Prepare input for combine
         taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.COMBINE, gridJob.id(), 0, 0, null);
 
-        GridHadoopTestTaskContext combineCtx = new GridHadoopTestTaskContext(taskInfo, gridJob);
+        HadoopTestTaskContext combineCtx = new HadoopTestTaskContext(taskInfo, gridJob);
 
         combineCtx.makeTreeOfWritables(mapCtx.mockOutput());
 
@@ -228,16 +228,16 @@ abstract class GridHadoopTasksAllVersionsTest extends GridHadoopAbstractWordCoun
         GridHadoopFileBlock fileBlock1 = new GridHadoopFileBlock(HOSTS, inFileUri, 0, l);
         GridHadoopFileBlock fileBlock2 = new GridHadoopFileBlock(HOSTS, inFileUri, l, fileLen - l);
 
-        GridHadoopV2Job gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopV2Job gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
 
-        GridHadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
+        HadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
 
-        GridHadoopTestTaskContext combine2Ctx = runMapCombineTask(fileBlock2, gridJob);
+        HadoopTestTaskContext combine2Ctx = runMapCombineTask(fileBlock2, gridJob);
 
         //Prepare input for combine
         GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.REDUCE, gridJob.id(), 0, 0, null);
 
-        GridHadoopTestTaskContext reduceCtx = new GridHadoopTestTaskContext(taskInfo, gridJob);
+        HadoopTestTaskContext reduceCtx = new HadoopTestTaskContext(taskInfo, gridJob);
 
         reduceCtx.makeTreeOfWritables(combine1Ctx.mockOutput());
         reduceCtx.makeTreeOfWritables(combine2Ctx.mockOutput());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
index c6b10bd..679be71 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV1Test.java
@@ -38,7 +38,7 @@ public class GridHadoopTasksV1Test extends GridHadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    @Override public GridHadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
         JobConf jobConf = GridHadoopWordCount1.getJob(inFile, outFile);
 
         setupFileSystems(jobConf);
@@ -47,7 +47,7 @@ public class GridHadoopTasksV1Test extends GridHadoopTasksAllVersionsTest {
 
         GridHadoopJobId jobId = new GridHadoopJobId(new UUID(0, 0), 0);
 
-        return new GridHadoopV2Job(jobId, jobInfo, log);
+        return new HadoopV2Job(jobId, jobInfo, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
index 13dd688..4d20b9c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTasksV2Test.java
@@ -42,7 +42,7 @@ public class GridHadoopTasksV2Test extends GridHadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws Exception if fails.
      */
-    @Override public GridHadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
         Job job = Job.getInstance();
 
         job.setOutputKeyClass(Text.class);
@@ -65,7 +65,7 @@ public class GridHadoopTasksV2Test extends GridHadoopTasksAllVersionsTest {
 
         GridHadoopJobId jobId = new GridHadoopJobId(new UUID(0, 0), 0);
 
-        return new GridHadoopV2Job(jobId, jobInfo, log);
+        return new HadoopV2Job(jobId, jobInfo, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
deleted file mode 100644
index 80b00a6..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestTaskContext.java
+++ /dev/null
@@ -1,219 +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.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Context for test purpose.
- */
-class GridHadoopTestTaskContext extends GridHadoopV2TaskContext {
-    /**
-     * Simple key-vale pair.
-     * @param <K> Key class.
-     * @param <V> Value class.
-     */
-    public static class Pair<K,V> {
-        /** Key */
-        private K key;
-
-        /** Value */
-        private V val;
-
-        /**
-         * @param key key.
-         * @param val value.
-         */
-        Pair(K key, V val) {
-            this.key = key;
-            this.val = val;
-        }
-
-        /**
-         * Getter of key.
-         * @return key.
-         */
-        K key() {
-            return key;
-        }
-
-        /**
-         * Getter of value.
-         * @return value.
-         */
-        V value() {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return key + "," + val;
-        }
-    }
-
-    /** Mock output container- result data of task execution if it is not overridden. */
-    private List<Pair<String, Integer>> mockOutput = new ArrayList<>();
-
-    /** Mock input container- input data if it is not overridden. */
-    private Map<Object,List> mockInput = new TreeMap<>();
-
-    /** Context output implementation to write data into mockOutput. */
-    private GridHadoopTaskOutput output = new GridHadoopTaskOutput() {
-        /** {@inheritDoc} */
-        @Override public void write(Object key, Object val) {
-            //Check of casting and extract/copy values
-            String strKey = new String(((Text)key).getBytes());
-            int intVal = ((IntWritable)val).get();
-
-            mockOutput().add(new Pair<>(strKey, intVal));
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() {
-            throw new UnsupportedOperationException();
-        }
-    };
-
-    /** Context input implementation to read data from mockInput. */
-    private GridHadoopTaskInput input = new GridHadoopTaskInput() {
-        /** Iterator of keys and associated lists of values. */
-        Iterator<Map.Entry<Object, List>> iter;
-
-        /** Current key and associated value list. */
-        Map.Entry<Object, List> currEntry;
-
-        /** {@inheritDoc} */
-        @Override public boolean next() {
-            if (iter == null)
-                iter = mockInput().entrySet().iterator();
-
-            if (iter.hasNext())
-                currEntry = iter.next();
-            else
-                currEntry = null;
-
-            return currEntry != null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object key() {
-            return currEntry.getKey();
-        }
-
-        /** {@inheritDoc} */
-        @Override public Iterator<?> values() {
-            return currEntry.getValue().iterator() ;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() {
-            throw new UnsupportedOperationException();
-        }
-    };
-
-    /**
-     * Getter of mock output container - result of task if it is not overridden.
-     *
-     * @return mock output.
-     */
-    public List<Pair<String, Integer>> mockOutput() {
-        return mockOutput;
-    }
-
-    /**
-     * Getter of mock input container- input data if it is not overridden.
-     *
-     * @return mock output.
-     */
-    public Map<Object, List> mockInput() {
-        return mockInput;
-    }
-
-    /**
-     * Generate one-key-multiple-values tree from array of key-value pairs, and wrap its into Writable objects.
-     * The result is placed into mock input.
-     *
-     * @param flatData list of key-value pair.
-     */
-    public void makeTreeOfWritables(Iterable<Pair<String, Integer>> flatData) {
-        Text key = new Text();
-
-        for (GridHadoopTestTaskContext.Pair<String, Integer> pair : flatData) {
-            key.set(pair.key);
-            ArrayList<IntWritable> valList;
-
-            if (!mockInput.containsKey(key)) {
-                valList = new ArrayList<>();
-                mockInput.put(key, valList);
-                key = new Text();
-            }
-            else
-                valList = (ArrayList<IntWritable>) mockInput.get(key);
-            valList.add(new IntWritable(pair.value()));
-        }
-    }
-
-    /**
-     * @param taskInfo Task info.
-     * @param gridJob Grid Hadoop job.
-     */
-    public GridHadoopTestTaskContext(GridHadoopTaskInfo taskInfo, GridHadoopJob gridJob) throws IgniteCheckedException {
-        super(taskInfo, gridJob, gridJob.id(), null, jobConfDataInput(gridJob));
-    }
-
-    /**
-     * Creates DataInput to read JobConf.
-     *
-     * @param job Job.
-     * @return DataInput with JobConf.
-     * @throws IgniteCheckedException If failed.
-     */
-    private static DataInput jobConfDataInput(GridHadoopJob job) throws IgniteCheckedException {
-        JobConf jobConf = new JobConf();
-
-        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)job.info()).properties().entrySet())
-            jobConf.set(e.getKey(), e.getValue());
-
-        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-        try {
-            jobConf.write(new DataOutputStream(buf));
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-
-        return new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskOutput output() {
-        return output;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskInput input() {
-        return input;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
deleted file mode 100644
index c7a456b..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopV2JobSelfTest.java
+++ /dev/null
@@ -1,88 +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.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.serializer.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
-
-/**
- * Self test of {@link GridHadoopV2Job}.
- */
-public class GridHadoopV2JobSelfTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static final String TEST_SERIALIZED_VALUE = "Test serialized value";
-
-    /**
-     * Custom serialization class that accepts {@link Writable}.
-     */
-    private static class CustomSerialization extends WritableSerialization {
-        /** {@inheritDoc} */
-        @Override public Deserializer<Writable> getDeserializer(Class<Writable> c) {
-            return new Deserializer<Writable>() {
-                @Override public void open(InputStream in) { }
-
-                @Override public Writable deserialize(Writable writable) {
-                    return new Text(TEST_SERIALIZED_VALUE);
-                }
-
-                @Override public void close() { }
-            };
-        }
-    }
-
-    /**
-     * Tests that {@link GridHadoopJob} provides wrapped serializer if it's set in configuration.
-     *
-     * @throws IgniteCheckedException If fails.
-     */
-    public void testCustomSerializationApplying() throws IgniteCheckedException {
-        JobConf cfg = new JobConf();
-
-        cfg.setMapOutputKeyClass(IntWritable.class);
-        cfg.setMapOutputValueClass(Text.class);
-        cfg.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
-
-        GridHadoopJob job = new GridHadoopV2Job(new GridHadoopJobId(UUID.randomUUID(), 1), createJobInfo(cfg), log);
-
-        GridHadoopTaskContext taskCtx = job.getTaskContext(new GridHadoopTaskInfo(GridHadoopTaskType.MAP, null, 0, 0,
-            null));
-
-        GridHadoopSerialization ser = taskCtx.keySerialization();
-
-        assertEquals(GridHadoopSerializationWrapper.class.getName(), ser.getClass().getName());
-
-        DataInput in = new DataInputStream(new ByteArrayInputStream(new byte[0]));
-
-        assertEquals(TEST_SERIALIZED_VALUE, ser.read(in, null).toString());
-
-        ser = taskCtx.valueSerialization();
-
-        assertEquals(GridHadoopSerializationWrapper.class.getName(), ser.getClass().getName());
-
-        assertEquals(TEST_SERIALIZED_VALUE, ser.read(in, null).toString());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java
new file mode 100644
index 0000000..116248f
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSerializationWrapperSelfTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hadoop.io.*;
+import org.apache.hadoop.io.serializer.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Test of wrapper of the native serialization.
+ */
+public class HadoopSerializationWrapperSelfTest extends GridCommonAbstractTest {
+    /**
+     * Tests read/write of IntWritable via native WritableSerialization.
+     * @throws Exception If fails.
+     */
+    public void testIntWritableSerialization() throws Exception {
+        GridHadoopSerialization ser = new HadoopSerializationWrapper(new WritableSerialization(), IntWritable.class);
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+        DataOutput out = new DataOutputStream(buf);
+
+        ser.write(out, new IntWritable(3));
+        ser.write(out, new IntWritable(-5));
+
+        assertEquals("[0, 0, 0, 3, -1, -1, -1, -5]", Arrays.toString(buf.toByteArray()));
+
+        DataInput in = new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
+
+        assertEquals(3, ((IntWritable)ser.read(in, null)).get());
+        assertEquals(-5, ((IntWritable)ser.read(in, null)).get());
+    }
+
+    /**
+     * Tests read/write of Integer via native JavaleSerialization.
+     * @throws Exception If fails.
+     */
+    public void testIntJavaSerialization() throws Exception {
+        GridHadoopSerialization ser = new HadoopSerializationWrapper(new JavaSerialization(), Integer.class);
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+        DataOutput out = new DataOutputStream(buf);
+
+        ser.write(out, 3);
+        ser.write(out, -5);
+        ser.close();
+
+        DataInput in = new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
+
+        assertEquals(3, ((Integer)ser.read(in, null)).intValue());
+        assertEquals(-5, ((Integer)ser.read(in, null)).intValue());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java
new file mode 100644
index 0000000..040730b
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapperSelfTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.lib.input.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+import org.apache.ignite.testframework.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Self test of {@link org.apache.ignite.internal.processors.hadoop.v2.HadoopSplitWrapper}.
+ */
+public class HadoopSplitWrapperSelfTest extends GridHadoopAbstractSelfTest {
+    /**
+     * Tests serialization of wrapper and the wrapped native split.
+     * @throws Exception If fails.
+     */
+    public void testSerialization() throws Exception {
+        FileSplit nativeSplit = new FileSplit(new Path("/path/to/file"), 100, 500, new String[]{"host1", "host2"});
+
+        assertEquals("/path/to/file:100+500", nativeSplit.toString());
+
+        HadoopSplitWrapper split = HadoopUtils.wrapSplit(10, nativeSplit, nativeSplit.getLocations());
+
+        assertEquals("[host1, host2]", Arrays.toString(split.hosts()));
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+        ObjectOutput out = new ObjectOutputStream(buf);
+
+        out.writeObject(split);
+
+        ObjectInput in = new ObjectInputStream(new ByteArrayInputStream(buf.toByteArray()));
+
+        final HadoopSplitWrapper res = (HadoopSplitWrapper)in.readObject();
+
+        assertEquals("/path/to/file:100+500", HadoopUtils.unwrapSplit(res).toString());
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                res.hosts();
+
+                return null;
+            }
+        }, AssertionError.class, null);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java
new file mode 100644
index 0000000..9b56300
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestTaskContext.java
@@ -0,0 +1,219 @@
+/*
+ * 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.hadoop.io.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Context for test purpose.
+ */
+class HadoopTestTaskContext extends HadoopV2TaskContext {
+    /**
+     * Simple key-vale pair.
+     * @param <K> Key class.
+     * @param <V> Value class.
+     */
+    public static class Pair<K,V> {
+        /** Key */
+        private K key;
+
+        /** Value */
+        private V val;
+
+        /**
+         * @param key key.
+         * @param val value.
+         */
+        Pair(K key, V val) {
+            this.key = key;
+            this.val = val;
+        }
+
+        /**
+         * Getter of key.
+         * @return key.
+         */
+        K key() {
+            return key;
+        }
+
+        /**
+         * Getter of value.
+         * @return value.
+         */
+        V value() {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return key + "," + val;
+        }
+    }
+
+    /** Mock output container- result data of task execution if it is not overridden. */
+    private List<Pair<String, Integer>> mockOutput = new ArrayList<>();
+
+    /** Mock input container- input data if it is not overridden. */
+    private Map<Object,List> mockInput = new TreeMap<>();
+
+    /** Context output implementation to write data into mockOutput. */
+    private GridHadoopTaskOutput output = new GridHadoopTaskOutput() {
+        /** {@inheritDoc} */
+        @Override public void write(Object key, Object val) {
+            //Check of casting and extract/copy values
+            String strKey = new String(((Text)key).getBytes());
+            int intVal = ((IntWritable)val).get();
+
+            mockOutput().add(new Pair<>(strKey, intVal));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            throw new UnsupportedOperationException();
+        }
+    };
+
+    /** Context input implementation to read data from mockInput. */
+    private GridHadoopTaskInput input = new GridHadoopTaskInput() {
+        /** Iterator of keys and associated lists of values. */
+        Iterator<Map.Entry<Object, List>> iter;
+
+        /** Current key and associated value list. */
+        Map.Entry<Object, List> currEntry;
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            if (iter == null)
+                iter = mockInput().entrySet().iterator();
+
+            if (iter.hasNext())
+                currEntry = iter.next();
+            else
+                currEntry = null;
+
+            return currEntry != null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object key() {
+            return currEntry.getKey();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<?> values() {
+            return currEntry.getValue().iterator() ;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            throw new UnsupportedOperationException();
+        }
+    };
+
+    /**
+     * Getter of mock output container - result of task if it is not overridden.
+     *
+     * @return mock output.
+     */
+    public List<Pair<String, Integer>> mockOutput() {
+        return mockOutput;
+    }
+
+    /**
+     * Getter of mock input container- input data if it is not overridden.
+     *
+     * @return mock output.
+     */
+    public Map<Object, List> mockInput() {
+        return mockInput;
+    }
+
+    /**
+     * Generate one-key-multiple-values tree from array of key-value pairs, and wrap its into Writable objects.
+     * The result is placed into mock input.
+     *
+     * @param flatData list of key-value pair.
+     */
+    public void makeTreeOfWritables(Iterable<Pair<String, Integer>> flatData) {
+        Text key = new Text();
+
+        for (HadoopTestTaskContext.Pair<String, Integer> pair : flatData) {
+            key.set(pair.key);
+            ArrayList<IntWritable> valList;
+
+            if (!mockInput.containsKey(key)) {
+                valList = new ArrayList<>();
+                mockInput.put(key, valList);
+                key = new Text();
+            }
+            else
+                valList = (ArrayList<IntWritable>) mockInput.get(key);
+            valList.add(new IntWritable(pair.value()));
+        }
+    }
+
+    /**
+     * @param taskInfo Task info.
+     * @param gridJob Grid Hadoop job.
+     */
+    public HadoopTestTaskContext(GridHadoopTaskInfo taskInfo, GridHadoopJob gridJob) throws IgniteCheckedException {
+        super(taskInfo, gridJob, gridJob.id(), null, jobConfDataInput(gridJob));
+    }
+
+    /**
+     * Creates DataInput to read JobConf.
+     *
+     * @param job Job.
+     * @return DataInput with JobConf.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static DataInput jobConfDataInput(GridHadoopJob job) throws IgniteCheckedException {
+        JobConf jobConf = new JobConf();
+
+        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)job.info()).properties().entrySet())
+            jobConf.set(e.getKey(), e.getValue());
+
+        ByteArrayOutputStream buf = new ByteArrayOutputStream();
+
+        try {
+            jobConf.write(new DataOutputStream(buf));
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        return new DataInputStream(new ByteArrayInputStream(buf.toByteArray()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskOutput output() {
+        return output;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskInput input() {
+        return input;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
new file mode 100644
index 0000000..66e35b5
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.hadoop.fs.*;
+import org.apache.hadoop.io.*;
+import org.apache.hadoop.io.serializer.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Self test of {@link org.apache.ignite.internal.processors.hadoop.v2.HadoopV2Job}.
+ */
+public class HadoopV2JobSelfTest extends GridHadoopAbstractSelfTest {
+    /** */
+    private static final String TEST_SERIALIZED_VALUE = "Test serialized value";
+
+    /**
+     * Custom serialization class that accepts {@link Writable}.
+     */
+    private static class CustomSerialization extends WritableSerialization {
+        /** {@inheritDoc} */
+        @Override public Deserializer<Writable> getDeserializer(Class<Writable> c) {
+            return new Deserializer<Writable>() {
+                @Override public void open(InputStream in) { }
+
+                @Override public Writable deserialize(Writable writable) {
+                    return new Text(TEST_SERIALIZED_VALUE);
+                }
+
+                @Override public void close() { }
+            };
+        }
+    }
+
+    /**
+     * Tests that {@link GridHadoopJob} provides wrapped serializer if it's set in configuration.
+     *
+     * @throws IgniteCheckedException If fails.
+     */
+    public void testCustomSerializationApplying() throws IgniteCheckedException {
+        JobConf cfg = new JobConf();
+
+        cfg.setMapOutputKeyClass(IntWritable.class);
+        cfg.setMapOutputValueClass(Text.class);
+        cfg.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
+
+        GridHadoopJob job = new HadoopV2Job(new GridHadoopJobId(UUID.randomUUID(), 1), createJobInfo(cfg), log);
+
+        GridHadoopTaskContext taskCtx = job.getTaskContext(new GridHadoopTaskInfo(GridHadoopTaskType.MAP, null, 0, 0,
+            null));
+
+        GridHadoopSerialization ser = taskCtx.keySerialization();
+
+        assertEquals(HadoopSerializationWrapper.class.getName(), ser.getClass().getName());
+
+        DataInput in = new DataInputStream(new ByteArrayInputStream(new byte[0]));
+
+        assertEquals(TEST_SERIALIZED_VALUE, ser.read(in, null).toString());
+
+        ser = taskCtx.valueSerialization();
+
+        assertEquals(HadoopSerializationWrapper.class.getName(), ser.getClass().getName());
+
+        assertEquals(TEST_SERIALIZED_VALUE, ser.read(in, null).toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopAbstractMapTest.java
index 716fe19..aa0ddc1 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopAbstractMapTest.java
@@ -60,12 +60,12 @@ public abstract class GridHadoopAbstractMapTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public GridHadoopSerialization keySerialization() throws IgniteCheckedException {
-            return new GridHadoopWritableSerialization(IntWritable.class);
+            return new HadoopWritableSerialization(IntWritable.class);
         }
 
         /** {@inheritDoc} */
         @Override public GridHadoopSerialization valueSerialization() throws IgniteCheckedException {
-            return new GridHadoopWritableSerialization(IntWritable.class);
+            return new HadoopWritableSerialization(IntWritable.class);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
deleted file mode 100644
index dd3c5d4..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
+++ /dev/null
@@ -1,209 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Tests Hadoop external communication component.
- */
-public class GridHadoopExternalCommunicationSelfTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleMessageSendingTcp() throws Exception {
-        checkSimpleMessageSending(false);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testSimpleMessageSendingShmem() throws Exception {
-        checkSimpleMessageSending(true);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkSimpleMessageSending(boolean useShmem) throws Exception {
-        UUID parentNodeId = UUID.randomUUID();
-
-        Marshaller marsh = new OptimizedMarshaller();
-
-        IgniteLogger log = log();
-
-        GridHadoopExternalCommunication[] comms = new GridHadoopExternalCommunication[4];
-
-        try {
-            String name = "grid";
-
-            TestHadoopListener[] lsnrs = new TestHadoopListener[4];
-
-            int msgs = 10;
-
-            for (int i = 0; i < comms.length; i++) {
-                comms[i] = new GridHadoopExternalCommunication(parentNodeId, UUID.randomUUID(), marsh, log,
-                    Executors.newFixedThreadPool(1), name + i);
-
-                if (useShmem)
-                    comms[i].setSharedMemoryPort(14000);
-
-                lsnrs[i] = new TestHadoopListener(msgs);
-
-                comms[i].setListener(lsnrs[i]);
-
-                comms[i].start();
-            }
-
-            for (int r = 0; r < msgs; r++) {
-                for (int from = 0; from < comms.length; from++) {
-                    for (int to = 0; to < comms.length; to++) {
-                        if (from == to)
-                            continue;
-
-                        comms[from].sendMessage(comms[to].localProcessDescriptor(), new TestMessage(from, to));
-                    }
-                }
-            }
-
-            U.sleep(1000);
-
-            for (TestHadoopListener lsnr : lsnrs) {
-                lsnr.await(3_000);
-
-                assertEquals(String.valueOf(lsnr.messages()), msgs * (comms.length - 1), lsnr.messages().size());
-            }
-        }
-        finally {
-            for (GridHadoopExternalCommunication comm : comms) {
-                if (comm != null)
-                    comm.stop();
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestHadoopListener implements GridHadoopMessageListener {
-        /** Received messages (array list is safe because executor has one thread). */
-        private Collection<TestMessage> msgs = new ArrayList<>();
-
-        /** Await latch. */
-        private CountDownLatch receiveLatch;
-
-        /**
-         * @param msgs Number of messages to await.
-         */
-        private TestHadoopListener(int msgs) {
-            receiveLatch = new CountDownLatch(msgs);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
-            assert msg instanceof TestMessage;
-
-            msgs.add((TestMessage)msg);
-
-            receiveLatch.countDown();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
-            // No-op.
-        }
-
-        /**
-         * @return Received messages.
-         */
-        public Collection<TestMessage> messages() {
-            return msgs;
-        }
-
-        /**
-         * @param millis Time to await.
-         * @throws InterruptedException If wait interrupted.
-         */
-        public void await(int millis) throws InterruptedException {
-            receiveLatch.await(millis, TimeUnit.MILLISECONDS);
-        }
-    }
-
-    /**
-     *
-     */
-    private static class TestMessage implements HadoopMessage {
-        /** From index. */
-        private int from;
-
-        /** To index. */
-        private int to;
-
-        /**
-         * @param from From index.
-         * @param to To index.
-         */
-        private TestMessage(int from, int to) {
-            this.from = from;
-            this.to = to;
-        }
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public TestMessage() {
-            // No-op.
-        }
-
-        /**
-         * @return From index.
-         */
-        public int from() {
-            return from;
-        }
-
-        /**
-         * @return To index.
-         */
-        public int to() {
-            return to;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeInt(from);
-            out.writeInt(to);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            from = in.readInt();
-            to = in.readInt();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java
new file mode 100644
index 0000000..a21633d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunicationSelfTest.java
@@ -0,0 +1,209 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Tests Hadoop external communication component.
+ */
+public class HadoopExternalCommunicationSelfTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleMessageSendingTcp() throws Exception {
+        checkSimpleMessageSending(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSimpleMessageSendingShmem() throws Exception {
+        checkSimpleMessageSending(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkSimpleMessageSending(boolean useShmem) throws Exception {
+        UUID parentNodeId = UUID.randomUUID();
+
+        Marshaller marsh = new OptimizedMarshaller();
+
+        IgniteLogger log = log();
+
+        HadoopExternalCommunication[] comms = new HadoopExternalCommunication[4];
+
+        try {
+            String name = "grid";
+
+            TestHadoopListener[] lsnrs = new TestHadoopListener[4];
+
+            int msgs = 10;
+
+            for (int i = 0; i < comms.length; i++) {
+                comms[i] = new HadoopExternalCommunication(parentNodeId, UUID.randomUUID(), marsh, log,
+                    Executors.newFixedThreadPool(1), name + i);
+
+                if (useShmem)
+                    comms[i].setSharedMemoryPort(14000);
+
+                lsnrs[i] = new TestHadoopListener(msgs);
+
+                comms[i].setListener(lsnrs[i]);
+
+                comms[i].start();
+            }
+
+            for (int r = 0; r < msgs; r++) {
+                for (int from = 0; from < comms.length; from++) {
+                    for (int to = 0; to < comms.length; to++) {
+                        if (from == to)
+                            continue;
+
+                        comms[from].sendMessage(comms[to].localProcessDescriptor(), new TestMessage(from, to));
+                    }
+                }
+            }
+
+            U.sleep(1000);
+
+            for (TestHadoopListener lsnr : lsnrs) {
+                lsnr.await(3_000);
+
+                assertEquals(String.valueOf(lsnr.messages()), msgs * (comms.length - 1), lsnr.messages().size());
+            }
+        }
+        finally {
+            for (HadoopExternalCommunication comm : comms) {
+                if (comm != null)
+                    comm.stop();
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestHadoopListener implements HadoopMessageListener {
+        /** Received messages (array list is safe because executor has one thread). */
+        private Collection<TestMessage> msgs = new ArrayList<>();
+
+        /** Await latch. */
+        private CountDownLatch receiveLatch;
+
+        /**
+         * @param msgs Number of messages to await.
+         */
+        private TestHadoopListener(int msgs) {
+            receiveLatch = new CountDownLatch(msgs);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
+            assert msg instanceof TestMessage;
+
+            msgs.add((TestMessage)msg);
+
+            receiveLatch.countDown();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
+            // No-op.
+        }
+
+        /**
+         * @return Received messages.
+         */
+        public Collection<TestMessage> messages() {
+            return msgs;
+        }
+
+        /**
+         * @param millis Time to await.
+         * @throws InterruptedException If wait interrupted.
+         */
+        public void await(int millis) throws InterruptedException {
+            receiveLatch.await(millis, TimeUnit.MILLISECONDS);
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestMessage implements HadoopMessage {
+        /** From index. */
+        private int from;
+
+        /** To index. */
+        private int to;
+
+        /**
+         * @param from From index.
+         * @param to To index.
+         */
+        private TestMessage(int from, int to) {
+            this.from = from;
+            this.to = to;
+        }
+
+        /**
+         * Required by {@link Externalizable}.
+         */
+        public TestMessage() {
+            // No-op.
+        }
+
+        /**
+         * @return From index.
+         */
+        public int from() {
+            return from;
+        }
+
+        /**
+         * @return To index.
+         */
+        public int to() {
+            return to;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(from);
+            out.writeInt(to);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            from = in.readInt();
+            to = in.readInt();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 822ab8f..1413c7e 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -94,10 +94,10 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopTaskExecutionSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopV2JobSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopV2JobSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSerializationWrapperSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSplitWrapperSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopSerializationWrapperSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopSplitWrapperSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopTasksV1Test.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopTasksV2Test.class.getName())));
@@ -107,7 +107,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopMapReduceEmbeddedSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopExternalTaskExecutionSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopExternalCommunicationSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopExternalCommunicationSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopSortingTest.class.getName())));
 


[20/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopAbstractCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopAbstractCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopAbstractCommunicationClient.java
new file mode 100644
index 0000000..3a5d84a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopAbstractCommunicationClient.java
@@ -0,0 +1,96 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.concurrent.atomic.*;
+
+/**
+ * Implements basic lifecycle for communication clients.
+ */
+public abstract class HadoopAbstractCommunicationClient implements HadoopCommunicationClient {
+    /** Time when this client was last used. */
+    private volatile long lastUsed = U.currentTimeMillis();
+
+    /** Reservations. */
+    private final AtomicInteger reserves = new AtomicInteger();
+
+    /** {@inheritDoc} */
+    @Override public boolean close() {
+        return reserves.compareAndSet(0, -1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void forceClose() {
+        reserves.set(-1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean closed() {
+        return reserves.get() == -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean reserve() {
+        while (true) {
+            int r = reserves.get();
+
+            if (r == -1)
+                return false;
+
+            if (reserves.compareAndSet(r, r + 1))
+                return true;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void release() {
+        while (true) {
+            int r = reserves.get();
+
+            if (r == -1)
+                return;
+
+            if (reserves.compareAndSet(r, r - 1))
+                return;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean reserved() {
+        return reserves.get() > 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getIdleTime() {
+        return U.currentTimeMillis() - lastUsed;
+    }
+
+    /**
+     * Updates used time.
+     */
+    protected void markUsed() {
+        lastUsed = U.currentTimeMillis();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopAbstractCommunicationClient.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopCommunicationClient.java
new file mode 100644
index 0000000..ce42e9a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopCommunicationClient.java
@@ -0,0 +1,72 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+
+/**
+ *
+ */
+public interface HadoopCommunicationClient {
+    /**
+     * @return {@code True} if client has been closed by this call,
+     *      {@code false} if failed to close client (due to concurrent reservation or concurrent close).
+     */
+    public boolean close();
+
+    /**
+     * Forces client close.
+     */
+    public void forceClose();
+
+    /**
+     * @return {@code True} if client is closed;
+     */
+    public boolean closed();
+
+    /**
+     * @return {@code True} if client was reserved, {@code false} otherwise.
+     */
+    public boolean reserve();
+
+    /**
+     * Releases this client by decreasing reservations.
+     */
+    public void release();
+
+    /**
+     * @return {@code True} if client was reserved.
+     */
+    public boolean reserved();
+
+    /**
+     * Gets idle time of this client.
+     *
+     * @return Idle time of this client.
+     */
+    public long getIdleTime();
+
+    /**
+     * @param desc Process descriptor.
+     * @param msg Message to send.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
new file mode 100644
index 0000000..4df8188
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -0,0 +1,1431 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.thread.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+import java.nio.*;
+import java.nio.channels.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Hadoop external communication class.
+ */
+public class HadoopExternalCommunication {
+    /** IPC error message. */
+    public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
+        "(switching to TCP, may be slower)."; // TODO IGNITE-70 Add link to documentation
+
+    /** Default port which node sets listener to (value is <tt>47100</tt>). */
+    public static final int DFLT_PORT = 27100;
+
+    /** Default connection timeout (value is <tt>1000</tt>ms). */
+    public static final long DFLT_CONN_TIMEOUT = 1000;
+
+    /** Default Maximum connection timeout (value is <tt>600,000</tt>ms). */
+    public static final long DFLT_MAX_CONN_TIMEOUT = 10 * 60 * 1000;
+
+    /** Default reconnect attempts count (value is <tt>10</tt>). */
+    public static final int DFLT_RECONNECT_CNT = 10;
+
+    /** Default message queue limit per connection (for incoming and outgoing . */
+    public static final int DFLT_MSG_QUEUE_LIMIT = GridNioServer.DFLT_SEND_QUEUE_LIMIT;
+
+    /**
+     * Default count of selectors for TCP server equals to
+     * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
+     */
+    public static final int DFLT_SELECTORS_CNT = 1;
+
+    /** Node ID meta for session. */
+    private static final int PROCESS_META = GridNioSessionMetaKey.nextUniqueKey();
+
+    /** Handshake timeout meta for session. */
+    private static final int HANDSHAKE_FINISH_META = GridNioSessionMetaKey.nextUniqueKey();
+
+    /** Message tracker meta for session. */
+    private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
+
+    /**
+     * Default local port range (value is <tt>100</tt>).
+     * See {@link #setLocalPortRange(int)} for details.
+     */
+    public static final int DFLT_PORT_RANGE = 100;
+
+    /** Default value for {@code TCP_NODELAY} socket option (value is <tt>true</tt>). */
+    public static final boolean DFLT_TCP_NODELAY = true;
+
+    /** Server listener. */
+    private final GridNioServerListener<HadoopMessage> srvLsnr =
+        new GridNioServerListenerAdapter<HadoopMessage>() {
+            @Override public void onConnected(GridNioSession ses) {
+                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+
+                assert desc != null : "Received connected notification without finished handshake: " + ses;
+            }
+
+            /** {@inheritDoc} */
+            @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
+                if (log.isDebugEnabled())
+                    log.debug("Closed connection for session: " + ses);
+
+                if (e != null)
+                    U.error(log, "Session disconnected due to exception: " + ses, e);
+
+                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+
+                if (desc != null) {
+                    HadoopCommunicationClient rmv = clients.remove(desc.processId());
+
+                    if (rmv != null)
+                        rmv.forceClose();
+                }
+
+                HadoopMessageListener lsnr0 = lsnr;
+
+                if (lsnr0 != null)
+                    // Notify listener about connection close.
+                    lsnr0.onConnectionLost(desc);
+            }
+
+            /** {@inheritDoc} */
+            @Override public void onMessage(GridNioSession ses, HadoopMessage msg) {
+                notifyListener(ses.<HadoopProcessDescriptor>meta(PROCESS_META), msg);
+
+                if (msgQueueLimit > 0) {
+                    GridNioMessageTracker tracker = ses.meta(TRACKER_META);
+
+                    assert tracker != null : "Missing tracker for limited message queue: " + ses;
+
+                    tracker.run();
+                }
+            }
+        };
+
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** Local process descriptor. */
+    private HadoopProcessDescriptor locProcDesc;
+
+    /** Marshaller. */
+    private Marshaller marsh;
+
+    /** Message notification executor service. */
+    private ExecutorService execSvc;
+
+    /** Grid name. */
+    private String gridName;
+
+    /** Complex variable that represents this node IP address. */
+    private volatile InetAddress locHost;
+
+    /** Local port which node uses. */
+    private int locPort = DFLT_PORT;
+
+    /** Local port range. */
+    private int locPortRange = DFLT_PORT_RANGE;
+
+    /** Local port which node uses to accept shared memory connections. */
+    private int shmemPort = -1;
+
+    /** Allocate direct buffer or heap buffer. */
+    private boolean directBuf = true;
+
+    /** Connect timeout. */
+    private long connTimeout = DFLT_CONN_TIMEOUT;
+
+    /** Maximum connect timeout. */
+    private long maxConnTimeout = DFLT_MAX_CONN_TIMEOUT;
+
+    /** Reconnect attempts count. */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+    private int reconCnt = DFLT_RECONNECT_CNT;
+
+    /** Socket send buffer. */
+    private int sockSndBuf;
+
+    /** Socket receive buffer. */
+    private int sockRcvBuf;
+
+    /** Message queue limit. */
+    private int msgQueueLimit = DFLT_MSG_QUEUE_LIMIT;
+
+    /** NIO server. */
+    private GridNioServer<HadoopMessage> nioSrvr;
+
+    /** Shared memory server. */
+    private IpcSharedMemoryServerEndpoint shmemSrv;
+
+    /** {@code TCP_NODELAY} option value for created sockets. */
+    private boolean tcpNoDelay = DFLT_TCP_NODELAY;
+
+    /** Shared memory accept worker. */
+    private ShmemAcceptWorker shmemAcceptWorker;
+
+    /** Shared memory workers. */
+    private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
+
+    /** Clients. */
+    private final ConcurrentMap<UUID, HadoopCommunicationClient> clients = GridConcurrentFactory.newMap();
+
+    /** Message listener. */
+    private volatile HadoopMessageListener lsnr;
+
+    /** Bound port. */
+    private int boundTcpPort = -1;
+
+    /** Bound port for shared memory server. */
+    private int boundTcpShmemPort = -1;
+
+    /** Count of selectors to use in TCP server. */
+    private int selectorsCnt = DFLT_SELECTORS_CNT;
+
+    /** Local node ID message. */
+    private ProcessHandshakeMessage locIdMsg;
+
+    /** Locks. */
+    private final GridKeyLock locks = new GridKeyLock();
+
+    /**
+     * @param parentNodeId Parent node ID.
+     * @param procId Process ID.
+     * @param marsh Marshaller to use.
+     * @param log Logger.
+     * @param execSvc Executor service for message notification.
+     * @param gridName Grid name.
+     */
+    public HadoopExternalCommunication(
+        UUID parentNodeId,
+        UUID procId,
+        Marshaller marsh,
+        IgniteLogger log,
+        ExecutorService execSvc,
+        String gridName
+    ) {
+        locProcDesc = new HadoopProcessDescriptor(parentNodeId, procId);
+
+        this.marsh = marsh;
+        this.log = log.getLogger(HadoopExternalCommunication.class);
+        this.execSvc = execSvc;
+        this.gridName = gridName;
+    }
+
+    /**
+     * Sets local port for socket binding.
+     * <p>
+     * If not provided, default value is {@link #DFLT_PORT}.
+     *
+     * @param locPort Port number.
+     */
+    public void setLocalPort(int locPort) {
+        this.locPort = locPort;
+    }
+
+    /**
+     * Gets local port for socket binding.
+     *
+     * @return Local port.
+     */
+    public int getLocalPort() {
+        return locPort;
+    }
+
+    /**
+     * Sets local port range for local host ports (value must greater than or equal to <tt>0</tt>).
+     * If provided local port (see {@link #setLocalPort(int)}} is occupied,
+     * implementation will try to increment the port number for as long as it is less than
+     * initial value plus this range.
+     * <p>
+     * If port range value is <tt>0</tt>, then implementation will try bind only to the port provided by
+     * {@link #setLocalPort(int)} method and fail if binding to this port did not succeed.
+     * <p>
+     * Local port range is very useful during development when more than one grid nodes need to run
+     * on the same physical machine.
+     * <p>
+     * If not provided, default value is {@link #DFLT_PORT_RANGE}.
+     *
+     * @param locPortRange New local port range.
+     */
+    public void setLocalPortRange(int locPortRange) {
+        this.locPortRange = locPortRange;
+    }
+
+    /**
+     * @return Local port range.
+     */
+    public int getLocalPortRange() {
+        return locPortRange;
+    }
+
+    /**
+     * Sets local port to accept shared memory connections.
+     * <p>
+     * If set to {@code -1} shared memory communication will be disabled.
+     * <p>
+     * If not provided, shared memory is disabled.
+     *
+     * @param shmemPort Port number.
+     */
+    public void setSharedMemoryPort(int shmemPort) {
+        this.shmemPort = shmemPort;
+    }
+
+    /**
+     * Gets shared memory port to accept incoming connections.
+     *
+     * @return Shared memory port.
+     */
+    public int getSharedMemoryPort() {
+        return shmemPort;
+    }
+
+    /**
+     * Sets connect timeout used when establishing connection
+     * with remote nodes.
+     * <p>
+     * {@code 0} is interpreted as infinite timeout.
+     * <p>
+     * If not provided, default value is {@link #DFLT_CONN_TIMEOUT}.
+     *
+     * @param connTimeout Connect timeout.
+     */
+    public void setConnectTimeout(long connTimeout) {
+        this.connTimeout = connTimeout;
+    }
+
+    /**
+     * @return Connection timeout.
+     */
+    public long getConnectTimeout() {
+        return connTimeout;
+    }
+
+    /**
+     * Sets maximum connect timeout. If handshake is not established within connect timeout,
+     * then SPI tries to repeat handshake procedure with increased connect timeout.
+     * Connect timeout can grow till maximum timeout value,
+     * if maximum timeout value is reached then the handshake is considered as failed.
+     * <p>
+     * {@code 0} is interpreted as infinite timeout.
+     * <p>
+     * If not provided, default value is {@link #DFLT_MAX_CONN_TIMEOUT}.
+     *
+     * @param maxConnTimeout Maximum connect timeout.
+     */
+    public void setMaxConnectTimeout(long maxConnTimeout) {
+        this.maxConnTimeout = maxConnTimeout;
+    }
+
+    /**
+     * Gets maximum connection timeout.
+     *
+     * @return Maximum connection timeout.
+     */
+    public long getMaxConnectTimeout() {
+        return maxConnTimeout;
+    }
+
+    /**
+     * Sets maximum number of reconnect attempts used when establishing connection
+     * with remote nodes.
+     * <p>
+     * If not provided, default value is {@link #DFLT_RECONNECT_CNT}.
+     *
+     * @param reconCnt Maximum number of reconnection attempts.
+     */
+    public void setReconnectCount(int reconCnt) {
+        this.reconCnt = reconCnt;
+    }
+
+    /**
+     * @return Reconnect count.
+     */
+    public int getReconnectCount() {
+        return reconCnt;
+    }
+
+    /**
+     * Sets flag to allocate direct or heap buffer in SPI.
+     * If value is {@code true}, then SPI will use {@link ByteBuffer#allocateDirect(int)} call.
+     * Otherwise, SPI will use {@link ByteBuffer#allocate(int)} call.
+     * <p>
+     * If not provided, default value is {@code true}.
+     *
+     * @param directBuf Flag indicates to allocate direct or heap buffer in SPI.
+     */
+    public void setDirectBuffer(boolean directBuf) {
+        this.directBuf = directBuf;
+    }
+
+    /**
+     * @return Direct buffer flag.
+     */
+    public boolean isDirectBuffer() {
+        return directBuf;
+    }
+
+    /**
+     * Sets the count of selectors te be used in TCP server.
+     * <p/>
+     * If not provided, default value is {@link #DFLT_SELECTORS_CNT}.
+     *
+     * @param selectorsCnt Selectors count.
+     */
+    public void setSelectorsCount(int selectorsCnt) {
+        this.selectorsCnt = selectorsCnt;
+    }
+
+    /**
+     * @return Number of selectors to use.
+     */
+    public int getSelectorsCount() {
+        return selectorsCnt;
+    }
+
+    /**
+     * Sets value for {@code TCP_NODELAY} socket option. Each
+     * socket will be opened using provided value.
+     * <p>
+     * Setting this option to {@code true} disables Nagle's algorithm
+     * for socket decreasing latency and delivery time for small messages.
+     * <p>
+     * For systems that work under heavy network load it is advisable to
+     * set this value to {@code false}.
+     * <p>
+     * If not provided, default value is {@link #DFLT_TCP_NODELAY}.
+     *
+     * @param tcpNoDelay {@code True} to disable TCP delay.
+     */
+    public void setTcpNoDelay(boolean tcpNoDelay) {
+        this.tcpNoDelay = tcpNoDelay;
+    }
+
+    /**
+     * @return {@code TCP_NO_DELAY} flag.
+     */
+    public boolean isTcpNoDelay() {
+        return tcpNoDelay;
+    }
+
+    /**
+     * Sets receive buffer size for sockets created or accepted by this SPI.
+     * <p>
+     * If not provided, default is {@code 0} which leaves buffer unchanged after
+     * socket creation (OS defaults).
+     *
+     * @param sockRcvBuf Socket receive buffer size.
+     */
+    public void setSocketReceiveBuffer(int sockRcvBuf) {
+        this.sockRcvBuf = sockRcvBuf;
+    }
+
+    /**
+     * @return Socket receive buffer size.
+     */
+    public int getSocketReceiveBuffer() {
+        return sockRcvBuf;
+    }
+
+    /**
+     * Sets send buffer size for sockets created or accepted by this SPI.
+     * <p>
+     * If not provided, default is {@code 0} which leaves the buffer unchanged
+     * after socket creation (OS defaults).
+     *
+     * @param sockSndBuf Socket send buffer size.
+     */
+    public void setSocketSendBuffer(int sockSndBuf) {
+        this.sockSndBuf = sockSndBuf;
+    }
+
+    /**
+     * @return Socket send buffer size.
+     */
+    public int getSocketSendBuffer() {
+        return sockSndBuf;
+    }
+
+    /**
+     * Sets message queue limit for incoming and outgoing messages.
+     * <p>
+     * When set to positive number send queue is limited to the configured value.
+     * {@code 0} disables the size limitations.
+     * <p>
+     * If not provided, default is {@link #DFLT_MSG_QUEUE_LIMIT}.
+     *
+     * @param msgQueueLimit Send queue size limit.
+     */
+    public void setMessageQueueLimit(int msgQueueLimit) {
+        this.msgQueueLimit = msgQueueLimit;
+    }
+
+    /**
+     * @return Message queue size limit.
+     */
+    public int getMessageQueueLimit() {
+        return msgQueueLimit;
+    }
+
+    /**
+     * Sets Hadoop communication message listener.
+     *
+     * @param lsnr Message listener.
+     */
+    public void setListener(HadoopMessageListener lsnr) {
+        this.lsnr = lsnr;
+    }
+
+    /**
+     * @return Outbound message queue size.
+     */
+    public int getOutboundMessagesQueueSize() {
+        return nioSrvr.outboundMessagesQueueSize();
+    }
+
+    /**
+     * Starts communication.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void start() throws IgniteCheckedException {
+        try {
+            locHost = U.getLocalHost();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to initialize local address.", e);
+        }
+
+        try {
+            shmemSrv = resetShmemServer();
+        }
+        catch (IgniteCheckedException e) {
+            U.warn(log, "Failed to start shared memory communication server.", e);
+        }
+
+        try {
+            // This method potentially resets local port to the value
+            // local node was bound to.
+            nioSrvr = resetNioServer();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Failed to initialize TCP server: " + locHost, e);
+        }
+
+        locProcDesc.address(locHost.getHostAddress());
+        locProcDesc.sharedMemoryPort(boundTcpShmemPort);
+        locProcDesc.tcpPort(boundTcpPort);
+
+        locIdMsg = new ProcessHandshakeMessage(locProcDesc);
+
+        if (shmemSrv != null) {
+            shmemAcceptWorker = new ShmemAcceptWorker(shmemSrv);
+
+            new IgniteThread(shmemAcceptWorker).start();
+        }
+
+        nioSrvr.start();
+    }
+
+    /**
+     * Gets local process descriptor.
+     *
+     * @return Local process descriptor.
+     */
+    public HadoopProcessDescriptor localProcessDescriptor() {
+        return locProcDesc;
+    }
+
+    /**
+     * Gets filters used by communication.
+     *
+     * @return Filters array.
+     */
+    private GridNioFilter[] filters() {
+        return new GridNioFilter[] {
+            new GridNioAsyncNotifyFilter(gridName, execSvc, log),
+            new HandshakeAndBackpressureFilter(),
+            new HadoopMarshallerFilter(marsh),
+            new GridNioCodecFilter(new GridBufferedParser(directBuf, ByteOrder.nativeOrder()), log, false)
+        };
+    }
+
+    /**
+     * Recreates tpcSrvr socket instance.
+     *
+     * @return Server instance.
+     * @throws IgniteCheckedException Thrown if it's not possible to create server.
+     */
+    private GridNioServer<HadoopMessage> resetNioServer() throws IgniteCheckedException {
+        if (boundTcpPort >= 0)
+            throw new IgniteCheckedException("Tcp NIO server was already created on port " + boundTcpPort);
+
+        IgniteCheckedException lastEx = null;
+
+        // If configured TCP port is busy, find first available in range.
+        for (int port = locPort; port < locPort + locPortRange; port++) {
+            try {
+                GridNioServer<HadoopMessage> srvr =
+                    GridNioServer.<HadoopMessage>builder()
+                        .address(locHost)
+                        .port(port)
+                        .listener(srvLsnr)
+                        .logger(log.getLogger(GridNioServer.class))
+                        .selectorCount(selectorsCnt)
+                        .gridName(gridName)
+                        .tcpNoDelay(tcpNoDelay)
+                        .directBuffer(directBuf)
+                        .byteOrder(ByteOrder.nativeOrder())
+                        .socketSendBufferSize(sockSndBuf)
+                        .socketReceiveBufferSize(sockRcvBuf)
+                        .sendQueueLimit(msgQueueLimit)
+                        .directMode(false)
+                        .filters(filters())
+                        .build();
+
+                boundTcpPort = port;
+
+                // Ack Port the TCP server was bound to.
+                if (log.isInfoEnabled())
+                    log.info("Successfully bound to TCP port [port=" + boundTcpPort +
+                        ", locHost=" + locHost + ']');
+
+                return srvr;
+            }
+            catch (IgniteCheckedException e) {
+                lastEx = e;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to bind to local port (will try next port within range) [port=" + port +
+                        ", locHost=" + locHost + ']');
+            }
+        }
+
+        // If free port wasn't found.
+        throw new IgniteCheckedException("Failed to bind to any port within range [startPort=" + locPort +
+            ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
+    }
+
+    /**
+     * Creates new shared memory communication server.
+     * @return Server.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable private IpcSharedMemoryServerEndpoint resetShmemServer() throws IgniteCheckedException {
+        if (boundTcpShmemPort >= 0)
+            throw new IgniteCheckedException("Shared memory server was already created on port " + boundTcpShmemPort);
+
+        if (shmemPort == -1 || U.isWindows())
+            return null;
+
+        IgniteCheckedException lastEx = null;
+
+        // If configured TCP port is busy, find first available in range.
+        for (int port = shmemPort; port < shmemPort + locPortRange; port++) {
+            try {
+                IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(
+                    log.getLogger(IpcSharedMemoryServerEndpoint.class),
+                    locProcDesc.processId(), gridName);
+
+                srv.setPort(port);
+
+                srv.omitOutOfResourcesWarning(true);
+
+                srv.start();
+
+                boundTcpShmemPort = port;
+
+                // Ack Port the TCP server was bound to.
+                if (log.isInfoEnabled())
+                    log.info("Successfully bound shared memory communication to TCP port [port=" + boundTcpShmemPort +
+                        ", locHost=" + locHost + ']');
+
+                return srv;
+            }
+            catch (IgniteCheckedException e) {
+                lastEx = e;
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to bind to local port (will try next port within range) [port=" + port +
+                        ", locHost=" + locHost + ']');
+            }
+        }
+
+        // If free port wasn't found.
+        throw new IgniteCheckedException("Failed to bind shared memory communication to any port within range [startPort=" +
+            locPort + ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
+    }
+
+    /**
+     * Stops the server.
+     *
+     * @throws IgniteCheckedException
+     */
+    public void stop() throws IgniteCheckedException {
+        // Stop TCP server.
+        if (nioSrvr != null)
+            nioSrvr.stop();
+
+        U.cancel(shmemAcceptWorker);
+        U.join(shmemAcceptWorker, log);
+
+        U.cancel(shmemWorkers);
+        U.join(shmemWorkers, log);
+
+        shmemWorkers.clear();
+
+        // Force closing on stop (safety).
+        for (HadoopCommunicationClient client : clients.values())
+            client.forceClose();
+
+        // Clear resources.
+        nioSrvr = null;
+
+        boundTcpPort = -1;
+    }
+
+    /**
+     * Sends message to Hadoop process.
+     *
+     * @param desc
+     * @param msg
+     * @throws IgniteCheckedException
+     */
+    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws
+        IgniteCheckedException {
+        assert desc != null;
+        assert msg != null;
+
+        if (log.isTraceEnabled())
+            log.trace("Sending message to Hadoop process [desc=" + desc + ", msg=" + msg + ']');
+
+        HadoopCommunicationClient client = null;
+
+        boolean closeOnRelease = true;
+
+        try {
+            client = reserveClient(desc);
+
+            client.sendMessage(desc, msg);
+
+            closeOnRelease = false;
+        }
+        finally {
+            if (client != null) {
+                if (closeOnRelease) {
+                    client.forceClose();
+
+                    clients.remove(desc.processId(), client);
+                }
+                else
+                    client.release();
+            }
+        }
+    }
+
+    /**
+     * Returns existing or just created client to node.
+     *
+     * @param desc Node to which client should be open.
+     * @return The existing or just created client.
+     * @throws IgniteCheckedException Thrown if any exception occurs.
+     */
+    private HadoopCommunicationClient reserveClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
+        assert desc != null;
+
+        UUID procId = desc.processId();
+
+        while (true) {
+            HadoopCommunicationClient client = clients.get(procId);
+
+            if (client == null) {
+                if (log.isDebugEnabled())
+                    log.debug("Did not find client for remote process [locProcDesc=" + locProcDesc + ", desc=" +
+                        desc + ']');
+
+                // Do not allow concurrent connects.
+                Object sync = locks.lock(procId);
+
+                try {
+                    client = clients.get(procId);
+
+                    if (client == null) {
+                        HadoopCommunicationClient old = clients.put(procId, client = createNioClient(desc));
+
+                        assert old == null;
+                    }
+                }
+                finally {
+                    locks.unlock(procId, sync);
+                }
+
+                assert client != null;
+            }
+
+            if (client.reserve())
+                return client;
+            else
+                // Client has just been closed by idle worker. Help it and try again.
+                clients.remove(procId, client);
+        }
+    }
+
+    /**
+     * @param desc Process descriptor.
+     * @return Client.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable protected HadoopCommunicationClient createNioClient(HadoopProcessDescriptor desc)
+        throws  IgniteCheckedException {
+        assert desc != null;
+
+        int shmemPort = desc.sharedMemoryPort();
+
+        // If remote node has shared memory server enabled and has the same set of MACs
+        // then we are likely to run on the same host and shared memory communication could be tried.
+        if (shmemPort != -1 && locProcDesc.parentNodeId().equals(desc.parentNodeId())) {
+            try {
+                return createShmemClient(desc, shmemPort);
+            }
+            catch (IgniteCheckedException e) {
+                if (e.hasCause(IpcOutOfSystemResourcesException.class))
+                    // Has cause or is itself the IpcOutOfSystemResourcesException.
+                    LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+                else if (log.isDebugEnabled())
+                    log.debug("Failed to establish shared memory connection with local hadoop process: " +
+                        desc);
+            }
+        }
+
+        return createTcpClient(desc);
+    }
+
+    /**
+     * @param desc Process descriptor.
+     * @param port Port.
+     * @return Client.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable protected HadoopCommunicationClient createShmemClient(HadoopProcessDescriptor desc, int port)
+        throws IgniteCheckedException {
+        int attempt = 1;
+
+        int connectAttempts = 1;
+
+        long connTimeout0 = connTimeout;
+
+        while (true) {
+            IpcEndpoint clientEndpoint;
+
+            try {
+                clientEndpoint = new IpcSharedMemoryClientEndpoint(port, (int)connTimeout, log);
+            }
+            catch (IgniteCheckedException e) {
+                // Reconnect for the second time, if connection is not established.
+                if (connectAttempts < 2 && X.hasCause(e, ConnectException.class)) {
+                    connectAttempts++;
+
+                    continue;
+                }
+
+                throw e;
+            }
+
+            HadoopCommunicationClient client = null;
+
+            try {
+                ShmemWorker worker = new ShmemWorker(clientEndpoint, false);
+
+                shmemWorkers.add(worker);
+
+                GridNioSession ses = worker.session();
+
+                HandshakeFinish fin = new HandshakeFinish();
+
+                // We are in lock, it is safe to get session and attach
+                ses.addMeta(HANDSHAKE_FINISH_META, fin);
+
+                client = new HadoopTcpNioCommunicationClient(ses);
+
+                new IgniteThread(worker).start();
+
+                fin.await(connTimeout0);
+            }
+            catch (HadoopHandshakeTimeoutException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Handshake timed out (will retry with increased timeout) [timeout=" + connTimeout0 +
+                        ", err=" + e.getMessage() + ", client=" + client + ']');
+
+                if (client != null)
+                    client.forceClose();
+
+                if (attempt == reconCnt || connTimeout0 > maxConnTimeout) {
+                    if (log.isDebugEnabled())
+                        log.debug("Handshake timedout (will stop attempts to perform the handshake) " +
+                            "[timeout=" + connTimeout0 + ", maxConnTimeout=" + maxConnTimeout +
+                            ", attempt=" + attempt + ", reconCnt=" + reconCnt +
+                            ", err=" + e.getMessage() + ", client=" + client + ']');
+
+                    throw e;
+                }
+                else {
+                    attempt++;
+
+                    connTimeout0 *= 2;
+
+                    continue;
+                }
+            }
+            catch (RuntimeException | Error e) {
+                if (log.isDebugEnabled())
+                    log.debug(
+                        "Caught exception (will close client) [err=" + e.getMessage() + ", client=" + client + ']');
+
+                if (client != null)
+                    client.forceClose();
+
+                throw e;
+            }
+
+            return client;
+        }
+    }
+
+    /**
+     * Establish TCP connection to remote hadoop process and returns client.
+     *
+     * @param desc Process descriptor.
+     * @return Client.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected HadoopCommunicationClient createTcpClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
+        String addr = desc.address();
+
+        int port = desc.tcpPort();
+
+        if (log.isDebugEnabled())
+            log.debug("Trying to connect to remote process [locProcDesc=" + locProcDesc + ", desc=" + desc + ']');
+
+        boolean conn = false;
+        HadoopTcpNioCommunicationClient client = null;
+        IgniteCheckedException errs = null;
+
+        int connectAttempts = 1;
+
+        long connTimeout0 = connTimeout;
+
+        int attempt = 1;
+
+        while (!conn) { // Reconnection on handshake timeout.
+            try {
+                SocketChannel ch = SocketChannel.open();
+
+                ch.configureBlocking(true);
+
+                ch.socket().setTcpNoDelay(tcpNoDelay);
+                ch.socket().setKeepAlive(true);
+
+                if (sockRcvBuf > 0)
+                    ch.socket().setReceiveBufferSize(sockRcvBuf);
+
+                if (sockSndBuf > 0)
+                    ch.socket().setSendBufferSize(sockSndBuf);
+
+                ch.socket().connect(new InetSocketAddress(addr, port), (int)connTimeout);
+
+                HandshakeFinish fin = new HandshakeFinish();
+
+                GridNioSession ses = nioSrvr.createSession(ch, F.asMap(HANDSHAKE_FINISH_META, fin)).get();
+
+                client = new HadoopTcpNioCommunicationClient(ses);
+
+                if (log.isDebugEnabled())
+                    log.debug("Waiting for handshake finish for client: " + client);
+
+                fin.await(connTimeout0);
+
+                conn = true;
+            }
+            catch (HadoopHandshakeTimeoutException e) {
+                if (client != null) {
+                    client.forceClose();
+
+                    client = null;
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug(
+                        "Handshake timedout (will retry with increased timeout) [timeout=" + connTimeout0 +
+                            ", desc=" + desc + ", port=" + port + ", err=" + e + ']');
+
+                if (attempt == reconCnt || connTimeout0 > maxConnTimeout) {
+                    if (log.isDebugEnabled())
+                        log.debug("Handshake timed out (will stop attempts to perform the handshake) " +
+                            "[timeout=" + connTimeout0 + ", maxConnTimeout=" + maxConnTimeout +
+                            ", attempt=" + attempt + ", reconCnt=" + reconCnt +
+                            ", err=" + e.getMessage() + ", addr=" + addr + ']');
+
+                    if (errs == null)
+                        errs = new IgniteCheckedException("Failed to connect to remote Hadoop process " +
+                            "(is process still running?) [desc=" + desc + ", addrs=" + addr + ']');
+
+                    errs.addSuppressed(e);
+
+                    break;
+                }
+                else {
+                    attempt++;
+
+                    connTimeout0 *= 2;
+
+                    // Continue loop.
+                }
+            }
+            catch (Exception e) {
+                if (client != null) {
+                    client.forceClose();
+
+                    client = null;
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Client creation failed [addr=" + addr + ", port=" + port +
+                        ", err=" + e + ']');
+
+                if (X.hasCause(e, SocketTimeoutException.class))
+                    LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
+                        "configuration property) [addr=" + addr + ", port=" + port + ']');
+
+                if (errs == null)
+                    errs = new IgniteCheckedException("Failed to connect to remote Hadoop process (is process still running?) " +
+                        "[desc=" + desc + ", addrs=" + addr + ']');
+
+                errs.addSuppressed(e);
+
+                // Reconnect for the second time, if connection is not established.
+                if (connectAttempts < 2 &&
+                    (e instanceof ConnectException || X.hasCause(e, ConnectException.class))) {
+                    connectAttempts++;
+
+                    continue;
+                }
+
+                break;
+            }
+        }
+
+        if (client == null) {
+            assert errs != null;
+
+            if (X.hasCause(errs, ConnectException.class))
+                LT.warn(log, null, "Failed to connect to a remote Hadoop process (is process still running?). " +
+                    "Make sure operating system firewall is disabled on local and remote host) " +
+                    "[addrs=" + addr + ", port=" + port + ']');
+
+            throw errs;
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Created client: " + client);
+
+        return client;
+    }
+
+    /**
+     * @param desc Sender process descriptor.
+     * @param msg Communication message.
+     */
+    protected void notifyListener(HadoopProcessDescriptor desc, HadoopMessage msg) {
+        HadoopMessageListener lsnr = this.lsnr;
+
+        if (lsnr != null)
+            // Notify listener of a new message.
+            lsnr.onMessageReceived(desc, msg);
+        else if (log.isDebugEnabled())
+            log.debug("Received communication message without any registered listeners (will ignore) " +
+                "[senderProcDesc=" + desc + ", msg=" + msg + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopExternalCommunication.class, this);
+    }
+
+    /**
+     * This worker takes responsibility to shut the server down when stopping,
+     * No other thread shall stop passed server.
+     */
+    private class ShmemAcceptWorker extends GridWorker {
+        /** */
+        private final IpcSharedMemoryServerEndpoint srv;
+
+        /**
+         * @param srv Server.
+         */
+        ShmemAcceptWorker(IpcSharedMemoryServerEndpoint srv) {
+            super(gridName, "shmem-communication-acceptor", log);
+
+            this.srv = srv;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            try {
+                while (!Thread.interrupted()) {
+                    ShmemWorker e = new ShmemWorker(srv.accept(), true);
+
+                    shmemWorkers.add(e);
+
+                    new IgniteThread(e).start();
+                }
+            }
+            catch (IgniteCheckedException e) {
+                if (!isCancelled())
+                    U.error(log, "Shmem server failed.", e);
+            }
+            finally {
+                srv.close();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            super.cancel();
+
+            srv.close();
+        }
+    }
+
+    /**
+     *
+     */
+    private class ShmemWorker extends GridWorker {
+        /** */
+        private final IpcEndpoint endpoint;
+
+        /** Adapter. */
+        private HadoopIpcToNioAdapter<HadoopMessage> adapter;
+
+        /**
+         * @param endpoint Endpoint.
+         */
+        private ShmemWorker(IpcEndpoint endpoint, boolean accepted) {
+            super(gridName, "shmem-worker", log);
+
+            this.endpoint = endpoint;
+
+            adapter = new HadoopIpcToNioAdapter<>(
+                HadoopExternalCommunication.this.log,
+                endpoint,
+                accepted,
+                srvLsnr,
+                filters());
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            try {
+                adapter.serve();
+            }
+            finally {
+                shmemWorkers.remove(this);
+
+                endpoint.close();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            super.cancel();
+
+            endpoint.close();
+        }
+
+        /** @{@inheritDoc} */
+        @Override protected void cleanup() {
+            super.cleanup();
+
+            endpoint.close();
+        }
+
+        /** @{@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ShmemWorker.class, this);
+        }
+
+        /**
+         * @return NIO session for this worker.
+         */
+        public GridNioSession session() {
+            return adapter.session();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class HandshakeFinish {
+        /** Await latch. */
+        private CountDownLatch latch = new CountDownLatch(1);
+
+        /**
+         * Finishes handshake.
+         */
+        public void finish() {
+            latch.countDown();
+        }
+
+        /**
+         * @param time Time to wait.
+         * @throws HadoopHandshakeTimeoutException If failed to wait.
+         */
+        public void await(long time) throws HadoopHandshakeTimeoutException {
+            try {
+                if (!latch.await(time, TimeUnit.MILLISECONDS))
+                    throw new HadoopHandshakeTimeoutException("Failed to wait for handshake to finish [timeout=" +
+                        time + ']');
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new HadoopHandshakeTimeoutException("Failed to wait for handshake to finish (thread was " +
+                    "interrupted) [timeout=" + time + ']', e);
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private class HandshakeAndBackpressureFilter extends GridNioFilterAdapter {
+        /**
+         * Assigns filter name to a filter.
+         */
+        protected HandshakeAndBackpressureFilter() {
+            super("HadoopHandshakeFilter");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionOpened(final GridNioSession ses) throws IgniteCheckedException {
+            if (ses.accepted()) {
+                if (log.isDebugEnabled())
+                    log.debug("Accepted connection, initiating handshake: " + ses);
+
+                // Server initiates handshake.
+                ses.send(locIdMsg).listenAsync(new CI1<GridNioFuture<?>>() {
+                    @Override public void apply(GridNioFuture<?> fut) {
+                        try {
+                            // Make sure there were no errors.
+                            fut.get();
+                        }
+                        catch (IgniteCheckedException | IOException e) {
+                            log.warning("Failed to send handshake message, will close session: " + ses, e);
+
+                            ses.close();
+                        }
+                    }
+                });
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionClosed(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+            proceedExceptionCaught(ses, ex);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+            if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
+                log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
+
+            return proceedSessionWrite(ses, msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
+            HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+
+            UUID rmtProcId = desc == null ? null : desc.processId();
+
+            if (rmtProcId == null) {
+                if (!(msg instanceof ProcessHandshakeMessage)) {
+                    log.warning("Invalid handshake message received, will close connection [ses=" + ses +
+                        ", msg=" + msg + ']');
+
+                    ses.close();
+
+                    return;
+                }
+
+                ProcessHandshakeMessage nId = (ProcessHandshakeMessage)msg;
+
+                if (log.isDebugEnabled())
+                    log.debug("Received handshake message [ses=" + ses + ", msg=" + msg + ']');
+
+                ses.addMeta(PROCESS_META, nId.processDescriptor());
+
+                if (!ses.accepted())
+                    // Send handshake reply.
+                    ses.send(locIdMsg);
+                else {
+                    //
+                    rmtProcId = nId.processDescriptor().processId();
+
+                    if (log.isDebugEnabled())
+                        log.debug("Finished handshake with remote client: " + ses);
+
+                    Object sync = locks.tryLock(rmtProcId);
+
+                    if (sync != null) {
+                        try {
+                            if (clients.get(rmtProcId) == null) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Will reuse session for descriptor: " + rmtProcId);
+
+                                // Handshake finished flag is true.
+                                clients.put(rmtProcId, new HadoopTcpNioCommunicationClient(ses));
+                            }
+                            else {
+                                if (log.isDebugEnabled())
+                                    log.debug("Will not reuse client as another already exists [locProcDesc=" +
+                                        locProcDesc + ", desc=" + desc + ']');
+                            }
+                        }
+                        finally {
+                            locks.unlock(rmtProcId, sync);
+                        }
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Concurrent connection is being established, will not reuse client session [" +
+                                "locProcDesc=" + locProcDesc + ", desc=" + desc + ']');
+                    }
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Handshake is finished for session [ses=" + ses + ", locProcDesc=" + locProcDesc + ']');
+
+                HandshakeFinish to = ses.meta(HANDSHAKE_FINISH_META);
+
+                if (to != null)
+                    to.finish();
+
+                // Notify session opened (both parties).
+                proceedSessionOpened(ses);
+            }
+            else {
+                if (msgQueueLimit > 0) {
+                    GridNioMessageTracker tracker = ses.meta(TRACKER_META);
+
+                    if (tracker == null) {
+                        GridNioMessageTracker old = ses.addMeta(TRACKER_META, tracker =
+                            new GridNioMessageTracker(ses, msgQueueLimit));
+
+                        assert old == null;
+                    }
+
+                    tracker.onMessageReceived();
+                }
+
+                proceedMessageReceived(ses, msg);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) throws IgniteCheckedException {
+            return proceedSessionClose(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionIdleTimeout(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionWriteTimeout(ses);
+        }
+    }
+
+    /**
+     * Process ID message.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class ProcessHandshakeMessage implements HadoopMessage {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Node ID. */
+        private HadoopProcessDescriptor procDesc;
+
+        /** */
+        public ProcessHandshakeMessage() {
+            // No-op.
+        }
+
+        /**
+         * @param procDesc Process descriptor.
+         */
+        private ProcessHandshakeMessage(HadoopProcessDescriptor procDesc) {
+            this.procDesc = procDesc;
+        }
+
+        /**
+         * @return Process ID.
+         */
+        public HadoopProcessDescriptor processDescriptor() {
+            return procDesc;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeObject(procDesc);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            procDesc = (HadoopProcessDescriptor)in.readObject();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ProcessHandshakeMessage.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopHandshakeTimeoutException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopHandshakeTimeoutException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopHandshakeTimeoutException.java
new file mode 100644
index 0000000..36cefcb
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopHandshakeTimeoutException.java
@@ -0,0 +1,42 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.jetbrains.annotations.*;
+
+/** Internal exception class for proper timeout handling. */
+class HadoopHandshakeTimeoutException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param msg Message.
+     */
+    HadoopHandshakeTimeoutException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * @param msg Message.
+     * @param cause Cause.
+     */
+    HadoopHandshakeTimeoutException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
new file mode 100644
index 0000000..8dbc96b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
@@ -0,0 +1,239 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.nio.*;
+
+import java.io.*;
+import java.nio.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Allows to re-use existing {@link GridNioFilter}s on IPC (specifically shared memory IPC)
+ * communications.
+ *
+ * Note that this class consumes an entire thread inside {@link #serve()} method
+ * in order to serve one {@link org.apache.ignite.internal.util.ipc.IpcEndpoint}.
+ */
+public class HadoopIpcToNioAdapter<T> {
+    /** */
+    private final IpcEndpoint endp;
+
+    /** */
+    private final GridNioFilterChain<T> chain;
+
+    /** */
+    private final GridNioSessionImpl ses;
+
+    /** */
+    private final AtomicReference<CountDownLatch> latchRef = new AtomicReference<>();
+
+    /** */
+    private final ByteBuffer writeBuf;
+
+    /**
+     * @param log Log.
+     * @param endp Endpoint.
+     * @param lsnr Listener.
+     * @param filters Filters.
+     */
+    public HadoopIpcToNioAdapter(IgniteLogger log, IpcEndpoint endp, boolean accepted,
+        GridNioServerListener<T> lsnr, GridNioFilter... filters) {
+        this.endp = endp;
+
+        chain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
+        ses = new GridNioSessionImpl(chain, null, null, accepted);
+
+        writeBuf = ByteBuffer.allocate(8 << 10);
+
+        writeBuf.order(ByteOrder.nativeOrder());
+    }
+
+    /**
+     * Serves given set of listeners repeatedly reading data from the endpoint.
+     *
+     * @throws InterruptedException If interrupted.
+     */
+    public void serve() throws InterruptedException {
+        try {
+            chain.onSessionOpened(ses);
+
+            InputStream in = endp.inputStream();
+
+            ByteBuffer readBuf = ByteBuffer.allocate(8 << 10);
+
+            readBuf.order(ByteOrder.nativeOrder());
+
+            assert readBuf.hasArray();
+
+            while (!Thread.interrupted()) {
+                int pos = readBuf.position();
+
+                int read = in.read(readBuf.array(), pos, readBuf.remaining());
+
+                if (read > 0) {
+                    readBuf.position(0);
+                    readBuf.limit(pos + read);
+
+                    chain.onMessageReceived(ses, readBuf);
+
+                    if (readBuf.hasRemaining())
+                        readBuf.compact();
+                    else
+                        readBuf.clear();
+
+                    CountDownLatch latch = latchRef.get();
+
+                    if (latch != null)
+                        latch.await();
+                }
+                else if (read < 0) {
+                    endp.close();
+
+                    break; // And close below.
+                }
+            }
+
+            // Assuming remote end closed connection - pushing event from head to tail.
+            chain.onSessionClosed(ses);
+        }
+        catch (Exception e) {
+            chain.onExceptionCaught(ses, new IgniteCheckedException("Failed to read from IPC endpoint.", e));
+        }
+    }
+
+    /**
+     * Gets dummy session for this adapter.
+     *
+     * @return Session.
+     */
+    public GridNioSession session() {
+        return ses;
+    }
+
+    /**
+     * Handles write events on chain.
+     *
+     * @param msg Buffer to send.
+     * @return Send result.
+     */
+    private GridNioFuture<?> send(ByteBuffer msg) {
+        assert writeBuf.hasArray();
+
+        try {
+            while (msg.hasRemaining()) {
+                writeBuf.clear();
+
+                writeBuf.put(msg);
+
+                endp.outputStream().write(writeBuf.array(), 0, writeBuf.position());
+            }
+        }
+        catch (IOException | IgniteCheckedException e) {
+            return new GridNioFinishedFuture<Object>(e);
+        }
+
+        return new GridNioFinishedFuture<>((Object)null);
+    }
+
+    /**
+     * Filter forwarding messages from chain's head to this server.
+     */
+    private class HeadFilter extends GridNioFilterAdapter {
+        /**
+         * Assigns filter name.
+         */
+        protected HeadFilter() {
+            super("HeadFilter");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionOpened(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionClosed(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+            proceedExceptionCaught(ses, ex);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+            assert ses == HadoopIpcToNioAdapter.this.ses : "ses=" + ses +
+                ", this.ses=" + HadoopIpcToNioAdapter.this.ses;
+
+            return send((ByteBuffer)msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
+            proceedMessageReceived(ses, msg);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<?> onPauseReads(GridNioSession ses) throws IgniteCheckedException {
+            // This call should be synced externally to avoid races.
+            boolean b = latchRef.compareAndSet(null, new CountDownLatch(1));
+
+            assert b;
+
+            return new GridNioFinishedFuture<>(b);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<?> onResumeReads(GridNioSession ses) throws IgniteCheckedException {
+            // This call should be synced externally to avoid races.
+            CountDownLatch latch = latchRef.getAndSet(null);
+
+            if (latch != null)
+                latch.countDown();
+
+            return new GridNioFinishedFuture<Object>(latch != null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) {
+            assert ses == HadoopIpcToNioAdapter.this.ses;
+
+            boolean closed = HadoopIpcToNioAdapter.this.ses.setClosed();
+
+            if (closed)
+                endp.close();
+
+            return new GridNioFinishedFuture<>(closed);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionIdleTimeout(ses);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
+            proceedSessionWriteTimeout(ses);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
new file mode 100644
index 0000000..66508a8
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
@@ -0,0 +1,84 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.marshaller.*;
+
+/**
+ * Serialization filter.
+ */
+public class HadoopMarshallerFilter extends GridNioFilterAdapter {
+    /** Marshaller. */
+    private Marshaller marshaller;
+
+    /**
+     * @param marshaller Marshaller to use.
+     */
+    public HadoopMarshallerFilter(Marshaller marshaller) {
+        super("GridHadoopMarshallerFilter");
+
+        this.marshaller = marshaller;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionOpened(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionClosed(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+        proceedExceptionCaught(ses, ex);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
+
+        return proceedSessionWrite(ses, marshaller.marshal(msg));
+    }
+
+    @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        assert msg instanceof byte[];
+
+        // Always unmarshal with system classloader.
+        proceedMessageReceived(ses, marshaller.unmarshal((byte[])msg, null));
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) throws IgniteCheckedException {
+        return proceedSessionClose(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionIdleTimeout(ses);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
+        proceedSessionWriteTimeout(ses);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMessageListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMessageListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMessageListener.java
new file mode 100644
index 0000000..c21e494
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMessageListener.java
@@ -0,0 +1,39 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+
+/**
+ * Hadoop communication message listener.
+ */
+public interface HadoopMessageListener {
+    /**
+     * @param desc Process descriptor.
+     * @param msg Hadoop message.
+     */
+    public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg);
+
+    /**
+     * Called when connection to remote process was lost.
+     *
+     * @param desc Process descriptor.
+     */
+    public void onConnectionLost(HadoopProcessDescriptor desc);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopTcpNioCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopTcpNioCommunicationClient.java
new file mode 100644
index 0000000..c4d1c54
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopTcpNioCommunicationClient.java
@@ -0,0 +1,99 @@
+/*
+ * 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.taskexecutor.external.communication;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Grid client for NIO server.
+ */
+public class HadoopTcpNioCommunicationClient extends HadoopAbstractCommunicationClient {
+    /** Socket. */
+    private final GridNioSession ses;
+
+    /**
+     * Constructor for test purposes only.
+     */
+    public HadoopTcpNioCommunicationClient() {
+        ses = null;
+    }
+
+    /**
+     * @param ses Session.
+     */
+    public HadoopTcpNioCommunicationClient(GridNioSession ses) {
+        assert ses != null;
+
+        this.ses = ses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean close() {
+        boolean res = super.close();
+
+        if (res)
+            ses.close();
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void forceClose() {
+        super.forceClose();
+
+        ses.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg)
+        throws IgniteCheckedException {
+        if (closed())
+            throw new IgniteCheckedException("Client was closed: " + this);
+
+        GridNioFuture<?> fut = ses.send(msg);
+
+        if (fut.isDone()) {
+            try {
+                fut.get();
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getIdleTime() {
+        long now = U.currentTimeMillis();
+
+        // Session can be used for receiving and sending.
+        return Math.min(Math.min(now - ses.lastReceiveTime(), now - ses.lastSendScheduleTime()),
+            now - ses.lastSendTime());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopTcpNioCommunicationClient.class, this, super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1CleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1CleanupTask.java
deleted file mode 100644
index 99ee9b77..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1CleanupTask.java
+++ /dev/null
@@ -1,62 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-import java.io.*;
-
-/**
- * Hadoop cleanup task implementation for v1 API.
- */
-public class GridHadoopV1CleanupTask extends GridHadoopV1Task {
-    /** Abort flag. */
-    private final boolean abort;
-
-    /**
-     * @param taskInfo Task info.
-     * @param abort Abort flag.
-     */
-    public GridHadoopV1CleanupTask(GridHadoopTaskInfo taskInfo, boolean abort) {
-        super(taskInfo);
-
-        this.abort = abort;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopV2TaskContext ctx = (GridHadoopV2TaskContext)taskCtx;
-
-        JobContext jobCtx = ctx.jobContext();
-
-        try {
-            OutputCommitter committer = jobCtx.getJobConf().getOutputCommitter();
-
-            if (abort)
-                committer.abortJob(jobCtx, JobStatus.State.FAILED);
-            else
-                committer.commitJob(jobCtx);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
deleted file mode 100644
index fb10720..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
+++ /dev/null
@@ -1,105 +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.v1;
-
-import org.apache.hadoop.mapred.Counters;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-import java.io.*;
-
-import static org.apache.hadoop.mapreduce.util.CountersStrings.*;
-
-/**
- * Hadoop counter implementation for v1 API.
- */
-public class GridHadoopV1Counter extends Counters.Counter {
-    /** Delegate. */
-    private final HadoopLongCounter cntr;
-
-    /**
-     * Creates new instance.
-     *
-     * @param cntr Delegate counter.
-     */
-    public GridHadoopV1Counter(HadoopLongCounter cntr) {
-        this.cntr = cntr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDisplayName(String displayName) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getName() {
-        return cntr.name();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getDisplayName() {
-        return getName();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getValue() {
-        return cntr.value();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setValue(long val) {
-        cntr.value(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void increment(long incr) {
-        cntr.increment(incr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(DataOutput out) throws IOException {
-        throw new UnsupportedOperationException("not implemented");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFields(DataInput in) throws IOException {
-        throw new UnsupportedOperationException("not implemented");
-    }
-
-    /** {@inheritDoc} */
-    @Override public String makeEscapedCompactString() {
-        return toEscapedCompactString(new GridHadoopV2Counter(cntr));
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public boolean contentEquals(Counters.Counter cntr) {
-        return getUnderlyingCounter().equals(cntr.getUnderlyingCounter());
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getCounter() {
-        return cntr.value();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counter getUnderlyingCounter() {
-        return this;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
deleted file mode 100644
index da59483..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
+++ /dev/null
@@ -1,111 +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.v1;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-/**
- * Hadoop map task implementation for v1 API.
- */
-public class GridHadoopV1MapTask extends GridHadoopV1Task {
-    /** */
-    private static final String[] EMPTY_HOSTS = new String[0];
-
-    /**
-     * Constructor.
-     *
-     * @param taskInfo 
-     */
-    public GridHadoopV1MapTask(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopJob job = taskCtx.job();
-
-        GridHadoopV2TaskContext ctx = (GridHadoopV2TaskContext)taskCtx;
-
-        JobConf jobConf = ctx.jobConf();
-
-        InputFormat inFormat = jobConf.getInputFormat();
-
-        GridHadoopInputSplit split = info().inputSplit();
-
-        InputSplit nativeSplit;
-
-        if (split instanceof GridHadoopFileBlock) {
-            GridHadoopFileBlock block = (GridHadoopFileBlock)split;
-
-            nativeSplit = new FileSplit(new Path(block.file().toString()), block.start(), block.length(), EMPTY_HOSTS);
-        }
-        else
-            nativeSplit = (InputSplit)ctx.getNativeSplit(split);
-
-        assert nativeSplit != null;
-
-        Reporter reporter = new GridHadoopV1Reporter(taskCtx);
-
-        GridHadoopV1OutputCollector collector = null;
-
-        try {
-            collector = collector(jobConf, ctx, !job.info().hasCombiner() && !job.info().hasReducer(),
-                fileName(), ctx.attemptId());
-
-            RecordReader reader = inFormat.getRecordReader(nativeSplit, jobConf, reporter);
-
-            Mapper mapper = ReflectionUtils.newInstance(jobConf.getMapperClass(), jobConf);
-
-            Object key = reader.createKey();
-            Object val = reader.createValue();
-
-            assert mapper != null;
-
-            try {
-                try {
-                    while (reader.next(key, val)) {
-                        if (isCancelled())
-                            throw new HadoopTaskCancelledException("Map task cancelled.");
-
-                        mapper.map(key, val, collector, reporter);
-                    }
-                }
-                finally {
-                    mapper.close();
-                }
-            }
-            finally {
-                collector.closeWriter();
-            }
-
-            collector.commit();
-        }
-        catch (Exception e) {
-            if (collector != null)
-                collector.abort();
-
-            throw new IgniteCheckedException(e);
-        }
-    }
-}


[19/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1OutputCollector.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1OutputCollector.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1OutputCollector.java
deleted file mode 100644
index 2a38684..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1OutputCollector.java
+++ /dev/null
@@ -1,130 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Hadoop output collector.
- */
-public class GridHadoopV1OutputCollector implements OutputCollector {
-    /** Job configuration. */
-    private final JobConf jobConf;
-
-    /** Task context. */
-    private final GridHadoopTaskContext taskCtx;
-
-    /** Optional direct writer. */
-    private final RecordWriter writer;
-
-    /** Task attempt. */
-    private final TaskAttemptID attempt;
-
-    /**
-     * @param jobConf Job configuration.
-     * @param taskCtx Task context.
-     * @param directWrite Direct write flag.
-     * @param fileName File name.
-     * @throws IOException In case of IO exception.
-     */
-    GridHadoopV1OutputCollector(JobConf jobConf, GridHadoopTaskContext taskCtx, boolean directWrite,
-        @Nullable String fileName, TaskAttemptID attempt) throws IOException {
-        this.jobConf = jobConf;
-        this.taskCtx = taskCtx;
-        this.attempt = attempt;
-
-        if (directWrite) {
-            jobConf.set("mapreduce.task.attempt.id", attempt.toString());
-
-            OutputFormat outFormat = jobConf.getOutputFormat();
-
-            writer = outFormat.getRecordWriter(null, jobConf, fileName, Reporter.NULL);
-        }
-        else
-            writer = null;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void collect(Object key, Object val) throws IOException {
-        if (writer != null)
-            writer.write(key, val);
-        else {
-            try {
-                taskCtx.output().write(key, val);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IOException(e);
-            }
-        }
-    }
-
-    /**
-     * Close writer.
-     *
-     * @throws IOException In case of IO exception.
-     */
-    public void closeWriter() throws IOException {
-        if (writer != null)
-            writer.close(Reporter.NULL);
-    }
-
-    /**
-     * Setup task.
-     *
-     * @throws IOException If failed.
-     */
-    public void setup() throws IOException {
-        if (writer != null)
-            jobConf.getOutputCommitter().setupTask(new TaskAttemptContextImpl(jobConf, attempt));
-    }
-
-    /**
-     * Commit task.
-     *
-     * @throws IOException In failed.
-     */
-    public void commit() throws IOException {
-        if (writer != null) {
-            OutputCommitter outputCommitter = jobConf.getOutputCommitter();
-
-            TaskAttemptContext taskCtx = new TaskAttemptContextImpl(jobConf, attempt);
-
-            if (outputCommitter.needsTaskCommit(taskCtx))
-                outputCommitter.commitTask(taskCtx);
-        }
-    }
-
-    /**
-     * Abort task.
-     */
-    public void abort() {
-        try {
-            if (writer != null)
-                jobConf.getOutputCommitter().abortTask(new TaskAttemptContextImpl(jobConf, attempt));
-        }
-        catch (IOException ignore) {
-            // No-op.
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Partitioner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Partitioner.java
deleted file mode 100644
index 688ccef..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Partitioner.java
+++ /dev/null
@@ -1,44 +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.v1;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-/**
- * Hadoop partitioner adapter for v1 API.
- */
-public class GridHadoopV1Partitioner implements GridHadoopPartitioner {
-    /** Partitioner instance. */
-    private Partitioner<Object, Object> part;
-
-    /**
-     * @param cls Hadoop partitioner class.
-     * @param conf Job configuration.
-     */
-    public GridHadoopV1Partitioner(Class<? extends Partitioner> cls, Configuration conf) {
-        part = (Partitioner<Object, Object>) ReflectionUtils.newInstance(cls, conf);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key, Object val, int parts) {
-        return part.getPartition(key, val, parts);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
deleted file mode 100644
index 3aca637..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
+++ /dev/null
@@ -1,92 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-/**
- * Hadoop reduce task implementation for v1 API.
- */
-public class GridHadoopV1ReduceTask extends GridHadoopV1Task {
-    /** {@code True} if reduce, {@code false} if combine. */
-    private final boolean reduce;
-
-    /**
-     * Constructor.
-     *
-     * @param taskInfo Task info.
-     * @param reduce {@code True} if reduce, {@code false} if combine.
-     */
-    public GridHadoopV1ReduceTask(GridHadoopTaskInfo taskInfo, boolean reduce) {
-        super(taskInfo);
-
-        this.reduce = reduce;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopJob job = taskCtx.job();
-
-        GridHadoopV2TaskContext ctx = (GridHadoopV2TaskContext)taskCtx;
-
-        JobConf jobConf = ctx.jobConf();
-
-        GridHadoopTaskInput input = taskCtx.input();
-
-        GridHadoopV1OutputCollector collector = null;
-
-        try {
-            collector = collector(jobConf, ctx, reduce || !job.info().hasReducer(), fileName(), ctx.attemptId());
-
-            Reducer reducer = ReflectionUtils.newInstance(reduce ? jobConf.getReducerClass() : jobConf.getCombinerClass(),
-                jobConf);
-
-            assert reducer != null;
-
-            try {
-                try {
-                    while (input.next()) {
-                        if (isCancelled())
-                            throw new HadoopTaskCancelledException("Reduce task cancelled.");
-
-                        reducer.reduce(input.key(), input.values(), collector, Reporter.NULL);
-                    }
-                }
-                finally {
-                    reducer.close();
-                }
-            }
-            finally {
-                collector.closeWriter();
-            }
-
-            collector.commit();
-        }
-        catch (Exception e) {
-            if (collector != null)
-                collector.abort();
-
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
deleted file mode 100644
index 791ccdc..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
+++ /dev/null
@@ -1,79 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-
-/**
- * Hadoop reporter implementation for v1 API.
- */
-public class GridHadoopV1Reporter implements Reporter {
-    /** Context. */
-    private final GridHadoopTaskContext ctx;
-
-    /**
-     * Creates new instance.
-     *
-     * @param ctx Context.
-     */
-    public GridHadoopV1Reporter(GridHadoopTaskContext ctx) {
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setStatus(String status) {
-        // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counters.Counter getCounter(Enum<?> name) {
-        return getCounter(name.getDeclaringClass().getName(), name.name());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counters.Counter getCounter(String grp, String name) {
-        return new GridHadoopV1Counter(ctx.counter(grp, name, HadoopLongCounter.class));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void incrCounter(Enum<?> key, long amount) {
-        getCounter(key).increment(amount);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void incrCounter(String grp, String cntr, long amount) {
-        getCounter(grp, cntr).increment(amount);
-    }
-
-    /** {@inheritDoc} */
-    @Override public InputSplit getInputSplit() throws UnsupportedOperationException {
-        throw new UnsupportedOperationException("reporter has no input"); // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getProgress() {
-        return 0.5f; // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public void progress() {
-        // TODO
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1SetupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1SetupTask.java
deleted file mode 100644
index c7dc3fd..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1SetupTask.java
+++ /dev/null
@@ -1,56 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-
-import java.io.*;
-
-/**
- * Hadoop setup task implementation for v1 API.
- */
-public class GridHadoopV1SetupTask extends GridHadoopV1Task {
-    /**
-     * Constructor.
-     *
-     * @param taskInfo Task info.
-     */
-    public GridHadoopV1SetupTask(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopV2TaskContext ctx = (GridHadoopV2TaskContext)taskCtx;
-
-        try {
-            ctx.jobConf().getOutputFormat().checkOutputSpecs(null, ctx.jobConf());
-
-            OutputCommitter committer = ctx.jobConf().getOutputCommitter();
-
-            if (committer != null)
-                committer.setupJob(ctx.jobContext());
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
deleted file mode 100644
index 0e1fb44..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
+++ /dev/null
@@ -1,97 +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.v1;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Hadoop API v1 splitter.
- */
-public class GridHadoopV1Splitter {
-    /** */
-    private static final String[] EMPTY_HOSTS = {};
-
-    /**
-     * @param jobConf Job configuration.
-     * @return Collection of mapped splits.
-     * @throws IgniteCheckedException If mapping failed.
-     */
-    public static Collection<GridHadoopInputSplit> splitJob(JobConf jobConf) throws IgniteCheckedException {
-        try {
-            InputFormat<?, ?> format = jobConf.getInputFormat();
-
-            assert format != null;
-
-            InputSplit[] splits = format.getSplits(jobConf, 0);
-
-            Collection<GridHadoopInputSplit> res = new ArrayList<>(splits.length);
-
-            for (int i = 0; i < splits.length; i++) {
-                InputSplit nativeSplit = splits[i];
-
-                if (nativeSplit instanceof FileSplit) {
-                    FileSplit s = (FileSplit)nativeSplit;
-
-                    res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
-                }
-                else
-                    res.add(HadoopUtils.wrapSplit(i, nativeSplit, nativeSplit.getLocations()));
-            }
-
-            return res;
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /**
-     * @param clsName Input split class name.
-     * @param in Input stream.
-     * @param hosts Optional hosts.
-     * @return File block or {@code null} if it is not a {@link FileSplit} instance.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public static GridHadoopFileBlock readFileBlock(String clsName, FSDataInputStream in,
-        @Nullable String[] hosts) throws IgniteCheckedException {
-        if (!FileSplit.class.getName().equals(clsName))
-            return null;
-
-        FileSplit split = U.newInstance(FileSplit.class);
-
-        try {
-            split.readFields(in);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-
-        if (hosts == null)
-            hosts = EMPTY_HOSTS;
-
-        return new GridHadoopFileBlock(hosts, split.getPath().toUri(), split.getStart(), split.getLength());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
deleted file mode 100644
index 305bc4e..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
+++ /dev/null
@@ -1,95 +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.v1;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.text.*;
-
-/**
- * Extended Hadoop v1 task.
- */
-public abstract class GridHadoopV1Task extends GridHadoopTask {
-    /** Indicates that this task is to be cancelled. */
-    private volatile boolean cancelled;
-
-    /**
-     * Constructor.
-     *
-     * @param taskInfo Task info.
-     */
-    protected GridHadoopV1Task(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /**
-     * Gets file name for that task result.
-     *
-     * @return File name.
-     */
-    public String fileName() {
-        NumberFormat numFormat = NumberFormat.getInstance();
-
-        numFormat.setMinimumIntegerDigits(5);
-        numFormat.setGroupingUsed(false);
-
-        return "part-" + numFormat.format(info().taskNumber());
-    }
-
-    /**
-     *
-     * @param jobConf Job configuration.
-     * @param taskCtx Task context.
-     * @param directWrite Direct write flag.
-     * @param fileName File name.
-     * @param attempt Attempt of task.
-     * @return Collector.
-     * @throws IOException In case of IO exception.
-     */
-    protected GridHadoopV1OutputCollector collector(JobConf jobConf, GridHadoopV2TaskContext taskCtx,
-        boolean directWrite, @Nullable String fileName, TaskAttemptID attempt) throws IOException {
-        GridHadoopV1OutputCollector collector = new GridHadoopV1OutputCollector(jobConf, taskCtx, directWrite,
-            fileName, attempt) {
-            /** {@inheritDoc} */
-            @Override public void collect(Object key, Object val) throws IOException {
-                if (cancelled)
-                    throw new HadoopTaskCancelledException("Task cancelled.");
-
-                super.collect(key, val);
-            }
-        };
-
-        collector.setup();
-
-        return collector;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cancel() {
-        cancelled = true;
-    }
-
-    /** Returns true if task is cancelled. */
-    public boolean isCancelled() {
-        return cancelled;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java
new file mode 100644
index 0000000..85f08be
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1CleanupTask.java
@@ -0,0 +1,62 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+
+/**
+ * Hadoop cleanup task implementation for v1 API.
+ */
+public class HadoopV1CleanupTask extends HadoopV1Task {
+    /** Abort flag. */
+    private final boolean abort;
+
+    /**
+     * @param taskInfo Task info.
+     * @param abort Abort flag.
+     */
+    public HadoopV1CleanupTask(GridHadoopTaskInfo taskInfo, boolean abort) {
+        super(taskInfo);
+
+        this.abort = abort;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
+
+        JobContext jobCtx = ctx.jobContext();
+
+        try {
+            OutputCommitter committer = jobCtx.getJobConf().getOutputCommitter();
+
+            if (abort)
+                committer.abortJob(jobCtx, JobStatus.State.FAILED);
+            else
+                committer.commitJob(jobCtx);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java
new file mode 100644
index 0000000..609297b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Counter.java
@@ -0,0 +1,105 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.Counters;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+
+import static org.apache.hadoop.mapreduce.util.CountersStrings.*;
+
+/**
+ * Hadoop counter implementation for v1 API.
+ */
+public class HadoopV1Counter extends Counters.Counter {
+    /** Delegate. */
+    private final HadoopLongCounter cntr;
+
+    /**
+     * Creates new instance.
+     *
+     * @param cntr Delegate counter.
+     */
+    public HadoopV1Counter(HadoopLongCounter cntr) {
+        this.cntr = cntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setDisplayName(String displayName) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getName() {
+        return cntr.name();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDisplayName() {
+        return getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getValue() {
+        return cntr.value();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setValue(long val) {
+        cntr.value(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void increment(long incr) {
+        cntr.increment(incr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(DataOutput out) throws IOException {
+        throw new UnsupportedOperationException("not implemented");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFields(DataInput in) throws IOException {
+        throw new UnsupportedOperationException("not implemented");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String makeEscapedCompactString() {
+        return toEscapedCompactString(new HadoopV2Counter(cntr));
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public boolean contentEquals(Counters.Counter cntr) {
+        return getUnderlyingCounter().equals(cntr.getUnderlyingCounter());
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCounter() {
+        return cntr.value();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counter getUnderlyingCounter() {
+        return this;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java
new file mode 100644
index 0000000..51856d6
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1MapTask.java
@@ -0,0 +1,111 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+/**
+ * Hadoop map task implementation for v1 API.
+ */
+public class HadoopV1MapTask extends HadoopV1Task {
+    /** */
+    private static final String[] EMPTY_HOSTS = new String[0];
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo 
+     */
+    public HadoopV1MapTask(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        GridHadoopJob job = taskCtx.job();
+
+        HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
+
+        JobConf jobConf = ctx.jobConf();
+
+        InputFormat inFormat = jobConf.getInputFormat();
+
+        GridHadoopInputSplit split = info().inputSplit();
+
+        InputSplit nativeSplit;
+
+        if (split instanceof GridHadoopFileBlock) {
+            GridHadoopFileBlock block = (GridHadoopFileBlock)split;
+
+            nativeSplit = new FileSplit(new Path(block.file().toString()), block.start(), block.length(), EMPTY_HOSTS);
+        }
+        else
+            nativeSplit = (InputSplit)ctx.getNativeSplit(split);
+
+        assert nativeSplit != null;
+
+        Reporter reporter = new HadoopV1Reporter(taskCtx);
+
+        HadoopV1OutputCollector collector = null;
+
+        try {
+            collector = collector(jobConf, ctx, !job.info().hasCombiner() && !job.info().hasReducer(),
+                fileName(), ctx.attemptId());
+
+            RecordReader reader = inFormat.getRecordReader(nativeSplit, jobConf, reporter);
+
+            Mapper mapper = ReflectionUtils.newInstance(jobConf.getMapperClass(), jobConf);
+
+            Object key = reader.createKey();
+            Object val = reader.createValue();
+
+            assert mapper != null;
+
+            try {
+                try {
+                    while (reader.next(key, val)) {
+                        if (isCancelled())
+                            throw new HadoopTaskCancelledException("Map task cancelled.");
+
+                        mapper.map(key, val, collector, reporter);
+                    }
+                }
+                finally {
+                    mapper.close();
+                }
+            }
+            finally {
+                collector.closeWriter();
+            }
+
+            collector.commit();
+        }
+        catch (Exception e) {
+            if (collector != null)
+                collector.abort();
+
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java
new file mode 100644
index 0000000..ac23bb3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1OutputCollector.java
@@ -0,0 +1,130 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Hadoop output collector.
+ */
+public class HadoopV1OutputCollector implements OutputCollector {
+    /** Job configuration. */
+    private final JobConf jobConf;
+
+    /** Task context. */
+    private final GridHadoopTaskContext taskCtx;
+
+    /** Optional direct writer. */
+    private final RecordWriter writer;
+
+    /** Task attempt. */
+    private final TaskAttemptID attempt;
+
+    /**
+     * @param jobConf Job configuration.
+     * @param taskCtx Task context.
+     * @param directWrite Direct write flag.
+     * @param fileName File name.
+     * @throws IOException In case of IO exception.
+     */
+    HadoopV1OutputCollector(JobConf jobConf, GridHadoopTaskContext taskCtx, boolean directWrite,
+        @Nullable String fileName, TaskAttemptID attempt) throws IOException {
+        this.jobConf = jobConf;
+        this.taskCtx = taskCtx;
+        this.attempt = attempt;
+
+        if (directWrite) {
+            jobConf.set("mapreduce.task.attempt.id", attempt.toString());
+
+            OutputFormat outFormat = jobConf.getOutputFormat();
+
+            writer = outFormat.getRecordWriter(null, jobConf, fileName, Reporter.NULL);
+        }
+        else
+            writer = null;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void collect(Object key, Object val) throws IOException {
+        if (writer != null)
+            writer.write(key, val);
+        else {
+            try {
+                taskCtx.output().write(key, val);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IOException(e);
+            }
+        }
+    }
+
+    /**
+     * Close writer.
+     *
+     * @throws IOException In case of IO exception.
+     */
+    public void closeWriter() throws IOException {
+        if (writer != null)
+            writer.close(Reporter.NULL);
+    }
+
+    /**
+     * Setup task.
+     *
+     * @throws IOException If failed.
+     */
+    public void setup() throws IOException {
+        if (writer != null)
+            jobConf.getOutputCommitter().setupTask(new TaskAttemptContextImpl(jobConf, attempt));
+    }
+
+    /**
+     * Commit task.
+     *
+     * @throws IOException In failed.
+     */
+    public void commit() throws IOException {
+        if (writer != null) {
+            OutputCommitter outputCommitter = jobConf.getOutputCommitter();
+
+            TaskAttemptContext taskCtx = new TaskAttemptContextImpl(jobConf, attempt);
+
+            if (outputCommitter.needsTaskCommit(taskCtx))
+                outputCommitter.commitTask(taskCtx);
+        }
+    }
+
+    /**
+     * Abort task.
+     */
+    public void abort() {
+        try {
+            if (writer != null)
+                jobConf.getOutputCommitter().abortTask(new TaskAttemptContextImpl(jobConf, attempt));
+        }
+        catch (IOException ignore) {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.java
new file mode 100644
index 0000000..36fdd55
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Partitioner.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.v1;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+/**
+ * Hadoop partitioner adapter for v1 API.
+ */
+public class HadoopV1Partitioner implements GridHadoopPartitioner {
+    /** Partitioner instance. */
+    private Partitioner<Object, Object> part;
+
+    /**
+     * @param cls Hadoop partitioner class.
+     * @param conf Job configuration.
+     */
+    public HadoopV1Partitioner(Class<? extends Partitioner> cls, Configuration conf) {
+        part = (Partitioner<Object, Object>) ReflectionUtils.newInstance(cls, conf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int partition(Object key, Object val, int parts) {
+        return part.getPartition(key, val, parts);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java
new file mode 100644
index 0000000..b5c6bfa
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1ReduceTask.java
@@ -0,0 +1,92 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+/**
+ * Hadoop reduce task implementation for v1 API.
+ */
+public class HadoopV1ReduceTask extends HadoopV1Task {
+    /** {@code True} if reduce, {@code false} if combine. */
+    private final boolean reduce;
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     * @param reduce {@code True} if reduce, {@code false} if combine.
+     */
+    public HadoopV1ReduceTask(GridHadoopTaskInfo taskInfo, boolean reduce) {
+        super(taskInfo);
+
+        this.reduce = reduce;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        GridHadoopJob job = taskCtx.job();
+
+        HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
+
+        JobConf jobConf = ctx.jobConf();
+
+        GridHadoopTaskInput input = taskCtx.input();
+
+        HadoopV1OutputCollector collector = null;
+
+        try {
+            collector = collector(jobConf, ctx, reduce || !job.info().hasReducer(), fileName(), ctx.attemptId());
+
+            Reducer reducer = ReflectionUtils.newInstance(reduce ? jobConf.getReducerClass() : jobConf.getCombinerClass(),
+                jobConf);
+
+            assert reducer != null;
+
+            try {
+                try {
+                    while (input.next()) {
+                        if (isCancelled())
+                            throw new HadoopTaskCancelledException("Reduce task cancelled.");
+
+                        reducer.reduce(input.key(), input.values(), collector, Reporter.NULL);
+                    }
+                }
+                finally {
+                    reducer.close();
+                }
+            }
+            finally {
+                collector.closeWriter();
+            }
+
+            collector.commit();
+        }
+        catch (Exception e) {
+            if (collector != null)
+                collector.abort();
+
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java
new file mode 100644
index 0000000..db4e159
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Reporter.java
@@ -0,0 +1,79 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+
+/**
+ * Hadoop reporter implementation for v1 API.
+ */
+public class HadoopV1Reporter implements Reporter {
+    /** Context. */
+    private final GridHadoopTaskContext ctx;
+
+    /**
+     * Creates new instance.
+     *
+     * @param ctx Context.
+     */
+    public HadoopV1Reporter(GridHadoopTaskContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setStatus(String status) {
+        // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counters.Counter getCounter(Enum<?> name) {
+        return getCounter(name.getDeclaringClass().getName(), name.name());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Counters.Counter getCounter(String grp, String name) {
+        return new HadoopV1Counter(ctx.counter(grp, name, HadoopLongCounter.class));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void incrCounter(Enum<?> key, long amount) {
+        getCounter(key).increment(amount);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void incrCounter(String grp, String cntr, long amount) {
+        getCounter(grp, cntr).increment(amount);
+    }
+
+    /** {@inheritDoc} */
+    @Override public InputSplit getInputSplit() throws UnsupportedOperationException {
+        throw new UnsupportedOperationException("reporter has no input"); // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getProgress() {
+        return 0.5f; // TODO
+    }
+
+    /** {@inheritDoc} */
+    @Override public void progress() {
+        // TODO
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.java
new file mode 100644
index 0000000..c427774
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1SetupTask.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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+
+import java.io.*;
+
+/**
+ * Hadoop setup task implementation for v1 API.
+ */
+public class HadoopV1SetupTask extends HadoopV1Task {
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     */
+    public HadoopV1SetupTask(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
+
+        try {
+            ctx.jobConf().getOutputFormat().checkOutputSpecs(null, ctx.jobConf());
+
+            OutputCommitter committer = ctx.jobConf().getOutputCommitter();
+
+            if (committer != null)
+                committer.setupJob(ctx.jobContext());
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java
new file mode 100644
index 0000000..0d89082
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Splitter.java
@@ -0,0 +1,97 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Hadoop API v1 splitter.
+ */
+public class HadoopV1Splitter {
+    /** */
+    private static final String[] EMPTY_HOSTS = {};
+
+    /**
+     * @param jobConf Job configuration.
+     * @return Collection of mapped splits.
+     * @throws IgniteCheckedException If mapping failed.
+     */
+    public static Collection<GridHadoopInputSplit> splitJob(JobConf jobConf) throws IgniteCheckedException {
+        try {
+            InputFormat<?, ?> format = jobConf.getInputFormat();
+
+            assert format != null;
+
+            InputSplit[] splits = format.getSplits(jobConf, 0);
+
+            Collection<GridHadoopInputSplit> res = new ArrayList<>(splits.length);
+
+            for (int i = 0; i < splits.length; i++) {
+                InputSplit nativeSplit = splits[i];
+
+                if (nativeSplit instanceof FileSplit) {
+                    FileSplit s = (FileSplit)nativeSplit;
+
+                    res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
+                }
+                else
+                    res.add(HadoopUtils.wrapSplit(i, nativeSplit, nativeSplit.getLocations()));
+            }
+
+            return res;
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /**
+     * @param clsName Input split class name.
+     * @param in Input stream.
+     * @param hosts Optional hosts.
+     * @return File block or {@code null} if it is not a {@link FileSplit} instance.
+     * @throws IgniteCheckedException If failed.
+     */
+    @Nullable public static GridHadoopFileBlock readFileBlock(String clsName, FSDataInputStream in,
+        @Nullable String[] hosts) throws IgniteCheckedException {
+        if (!FileSplit.class.getName().equals(clsName))
+            return null;
+
+        FileSplit split = U.newInstance(FileSplit.class);
+
+        try {
+            split.readFields(in);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        if (hosts == null)
+            hosts = EMPTY_HOSTS;
+
+        return new GridHadoopFileBlock(hosts, split.getPath().toUri(), split.getStart(), split.getLength());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java
new file mode 100644
index 0000000..71a259c
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/HadoopV1Task.java
@@ -0,0 +1,95 @@
+/*
+ * 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.v1;
+
+import org.apache.hadoop.mapred.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.text.*;
+
+/**
+ * Extended Hadoop v1 task.
+ */
+public abstract class HadoopV1Task extends GridHadoopTask {
+    /** Indicates that this task is to be cancelled. */
+    private volatile boolean cancelled;
+
+    /**
+     * Constructor.
+     *
+     * @param taskInfo Task info.
+     */
+    protected HadoopV1Task(GridHadoopTaskInfo taskInfo) {
+        super(taskInfo);
+    }
+
+    /**
+     * Gets file name for that task result.
+     *
+     * @return File name.
+     */
+    public String fileName() {
+        NumberFormat numFormat = NumberFormat.getInstance();
+
+        numFormat.setMinimumIntegerDigits(5);
+        numFormat.setGroupingUsed(false);
+
+        return "part-" + numFormat.format(info().taskNumber());
+    }
+
+    /**
+     *
+     * @param jobConf Job configuration.
+     * @param taskCtx Task context.
+     * @param directWrite Direct write flag.
+     * @param fileName File name.
+     * @param attempt Attempt of task.
+     * @return Collector.
+     * @throws IOException In case of IO exception.
+     */
+    protected HadoopV1OutputCollector collector(JobConf jobConf, HadoopV2TaskContext taskCtx,
+        boolean directWrite, @Nullable String fileName, TaskAttemptID attempt) throws IOException {
+        HadoopV1OutputCollector collector = new HadoopV1OutputCollector(jobConf, taskCtx, directWrite,
+            fileName, attempt) {
+            /** {@inheritDoc} */
+            @Override public void collect(Object key, Object val) throws IOException {
+                if (cancelled)
+                    throw new HadoopTaskCancelledException("Task cancelled.");
+
+                super.collect(key, val);
+            }
+        };
+
+        collector.setup();
+
+        return collector;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancel() {
+        cancelled = true;
+    }
+
+    /** Returns true if task is cancelled. */
+    public boolean isCancelled() {
+        return cancelled;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopExternalSplit.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopExternalSplit.java
deleted file mode 100644
index 36b40a2..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopExternalSplit.java
+++ /dev/null
@@ -1,87 +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.v2;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-
-import java.io.*;
-
-/**
- * Split serialized in external file.
- */
-public class GridHadoopExternalSplit extends GridHadoopInputSplit {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private long off;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public GridHadoopExternalSplit() {
-        // No-op.
-    }
-
-    /**
-     * @param hosts Hosts.
-     * @param off Offset of this split in external file.
-     */
-    public GridHadoopExternalSplit(String[] hosts, long off) {
-        assert off >= 0 : off;
-        assert hosts != null;
-
-        this.hosts = hosts;
-        this.off = off;
-    }
-
-    /**
-     * @return Offset of this input split in external file.
-     */
-    public long offset() {
-        return off;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeLong(off);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        off = in.readLong();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GridHadoopExternalSplit that = (GridHadoopExternalSplit) o;
-
-        return off == that.off;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return (int)(off ^ (off >>> 32));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopNativeCodeLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopNativeCodeLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopNativeCodeLoader.java
deleted file mode 100644
index 5ef4759..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopNativeCodeLoader.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.v2;
-
-import org.apache.hadoop.classification.*;
-import org.apache.hadoop.conf.*;
-
-/**
- * A fake helper to load the native hadoop code i.e. libhadoop.so.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class GridHadoopNativeCodeLoader {
-    /**
-     * Check if native-hadoop code is loaded for this platform.
-     *
-     * @return <code>true</code> if native-hadoop is loaded,
-     *         else <code>false</code>
-     */
-    public static boolean isNativeCodeLoaded() {
-        return false;
-    }
-
-    /**
-     * Returns true only if this build was compiled with support for snappy.
-     */
-    public static boolean buildSupportsSnappy() {
-        return false;
-    }
-
-    /**
-     * @return Library name.
-     */
-    public static String getLibraryName() {
-        throw new IllegalStateException();
-    }
-
-    /**
-     * Return if native hadoop libraries, if present, can be used for this job.
-     * @param conf configuration
-     *
-     * @return <code>true</code> if native hadoop libraries, if present, can be
-     *         used for this job; <code>false</code> otherwise.
-     */
-    public boolean getLoadNativeLibraries(Configuration conf) {
-        return false;
-    }
-
-    /**
-     * Set if native hadoop libraries, if present, can be used for this job.
-     *
-     * @param conf configuration
-     * @param loadNativeLibraries can native hadoop libraries be loaded
-     */
-    public void setLoadNativeLibraries(Configuration conf, boolean loadNativeLibraries) {
-        // No-op.
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSerializationWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSerializationWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSerializationWrapper.java
deleted file mode 100644
index 0f38548..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSerializationWrapper.java
+++ /dev/null
@@ -1,133 +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.v2;
-
-import org.apache.hadoop.io.serializer.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * The wrapper around external serializer.
- */
-public class GridHadoopSerializationWrapper<T> implements GridHadoopSerialization {
-    /** External serializer - writer. */
-    private final Serializer<T> serializer;
-
-    /** External serializer - reader. */
-    private final Deserializer<T> deserializer;
-
-    /** Data output for current write operation. */
-    private OutputStream currOut;
-
-    /** Data input for current read operation. */
-    private InputStream currIn;
-
-    /** Wrapper around current output to provide OutputStream interface. */
-    private final OutputStream outStream = new OutputStream() {
-        /** {@inheritDoc} */
-        @Override public void write(int b) throws IOException {
-            currOut.write(b);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(byte[] b, int off, int len) throws IOException {
-            currOut.write(b, off, len);
-        }
-    };
-
-    /** Wrapper around current input to provide InputStream interface. */
-    private final InputStream inStream = new InputStream() {
-        /** {@inheritDoc} */
-        @Override public int read() throws IOException {
-            return currIn.read();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int read(byte[] b, int off, int len) throws IOException {
-            return currIn.read(b, off, len);
-        }
-    };
-
-    /**
-     * @param serialization External serializer to wrap.
-     * @param cls The class to serialize.
-     */
-    public GridHadoopSerializationWrapper(Serialization<T> serialization, Class<T> cls) throws IgniteCheckedException {
-        assert cls != null;
-
-        serializer = serialization.getSerializer(cls);
-        deserializer = serialization.getDeserializer(cls);
-
-        try {
-            serializer.open(outStream);
-            deserializer.open(inStream);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(DataOutput out, Object obj) throws IgniteCheckedException {
-        assert out != null;
-        assert obj != null;
-
-        try {
-            currOut = (OutputStream)out;
-
-            serializer.serialize((T)obj);
-
-            currOut = null;
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object read(DataInput in, @Nullable Object obj) throws IgniteCheckedException {
-        assert in != null;
-
-        try {
-            currIn = (InputStream)in;
-
-            T res = deserializer.deserialize((T) obj);
-
-            currIn = null;
-
-            return res;
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
-        try {
-            serializer.close();
-            deserializer.close();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopShutdownHookManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopShutdownHookManager.java
deleted file mode 100644
index 48558fc..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopShutdownHookManager.java
+++ /dev/null
@@ -1,96 +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.v2;
-
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Fake manager for shutdown hooks.
- */
-public class GridHadoopShutdownHookManager {
-    /** */
-    private static final GridHadoopShutdownHookManager MGR = new GridHadoopShutdownHookManager();
-
-    /**
-     * Return <code>ShutdownHookManager</code> singleton.
-     *
-     * @return <code>ShutdownHookManager</code> singleton.
-     */
-    public static GridHadoopShutdownHookManager get() {
-        return MGR;
-    }
-
-    /** */
-    private Set<Runnable> hooks = Collections.synchronizedSet(new HashSet<Runnable>());
-
-    /** */
-    private AtomicBoolean shutdownInProgress = new AtomicBoolean(false);
-
-    /**
-     * Singleton.
-     */
-    private GridHadoopShutdownHookManager() {
-        // No-op.
-    }
-
-    /**
-     * Adds a shutdownHook with a priority, the higher the priority
-     * the earlier will run. ShutdownHooks with same priority run
-     * in a non-deterministic order.
-     *
-     * @param shutdownHook shutdownHook <code>Runnable</code>
-     * @param priority priority of the shutdownHook.
-     */
-    public void addShutdownHook(Runnable shutdownHook, int priority) {
-        if (shutdownHook == null)
-            throw new IllegalArgumentException("shutdownHook cannot be NULL");
-
-        hooks.add(shutdownHook);
-    }
-
-    /**
-     * Removes a shutdownHook.
-     *
-     * @param shutdownHook shutdownHook to remove.
-     * @return TRUE if the shutdownHook was registered and removed,
-     * FALSE otherwise.
-     */
-    public boolean removeShutdownHook(Runnable shutdownHook) {
-        return hooks.remove(shutdownHook);
-    }
-
-    /**
-     * Indicates if a shutdownHook is registered or not.
-     *
-     * @param shutdownHook shutdownHook to check if registered.
-     * @return TRUE/FALSE depending if the shutdownHook is is registered.
-     */
-    public boolean hasShutdownHook(Runnable shutdownHook) {
-        return hooks.contains(shutdownHook);
-    }
-
-    /**
-     * Indicates if shutdown is in progress or not.
-     *
-     * @return TRUE if the shutdown is in progress, otherwise FALSE.
-     */
-    public boolean isShutdownInProgress() {
-        return shutdownInProgress.get();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSplitWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSplitWrapper.java
deleted file mode 100644
index 57edfa9..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopSplitWrapper.java
+++ /dev/null
@@ -1,118 +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.v2;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * The wrapper for native hadoop input splits.
- *
- * Warning!! This class must not depend on any Hadoop classes directly or indirectly.
- */
-public class GridHadoopSplitWrapper extends GridHadoopInputSplit {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Native hadoop input split. */
-    private byte[] bytes;
-
-    /** */
-    private String clsName;
-
-    /** Internal ID */
-    private int id;
-
-    /**
-     * Creates new split wrapper.
-     */
-    public GridHadoopSplitWrapper() {
-        // No-op.
-    }
-
-    /**
-     * Creates new split wrapper.
-     *
-     * @param id Split ID.
-     * @param clsName Class name.
-     * @param bytes Serialized class.
-     * @param hosts Hosts where split is located.
-     */
-    public GridHadoopSplitWrapper(int id, String clsName, byte[] bytes, String[] hosts) {
-        assert hosts != null;
-        assert clsName != null;
-        assert bytes != null;
-
-        this.hosts = hosts;
-        this.id = id;
-
-        this.clsName = clsName;
-        this.bytes = bytes;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(id);
-
-        out.writeUTF(clsName);
-        U.writeByteArray(out, bytes);
-    }
-
-    /**
-     * @return Class name.
-     */
-    public String className() {
-        return clsName;
-    }
-
-    /**
-     * @return Class bytes.
-     */
-    public byte[] bytes() {
-        return bytes;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        id = in.readInt();
-
-        clsName = in.readUTF();
-        bytes = U.readByteArray(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GridHadoopSplitWrapper that = (GridHadoopSplitWrapper)o;
-
-        return id == that.id;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return id;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2CleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2CleanupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2CleanupTask.java
deleted file mode 100644
index 38be3da..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2CleanupTask.java
+++ /dev/null
@@ -1,73 +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.v2;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.JobStatus;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.*;
-
-import java.io.*;
-
-/**
- * Hadoop cleanup task (commits or aborts job).
- */
-public class GridHadoopV2CleanupTask extends GridHadoopV2Task {
-    /** Abort flag. */
-    private final boolean abort;
-
-    /**
-     * @param taskInfo Task info.
-     * @param abort Abort flag.
-     */
-    public GridHadoopV2CleanupTask(GridHadoopTaskInfo taskInfo, boolean abort) {
-        super(taskInfo);
-
-        this.abort = abort;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("ConstantConditions")
-    @Override public void run0(GridHadoopV2TaskContext taskCtx) throws IgniteCheckedException {
-        JobContextImpl jobCtx = taskCtx.jobContext();
-
-        try {
-            OutputFormat outputFormat = getOutputFormat(jobCtx);
-
-            OutputCommitter committer = outputFormat.getOutputCommitter(hadoopContext());
-
-            if (committer != null) {
-                if (abort)
-                    committer.abortJob(jobCtx, JobStatus.State.FAILED);
-                else
-                    committer.commitJob(jobCtx);
-            }
-        }
-        catch (ClassNotFoundException | IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
deleted file mode 100644
index 9964d91..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
+++ /dev/null
@@ -1,230 +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.v2;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.mapreduce.task.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Hadoop context implementation for v2 API. It provides IO operations for hadoop tasks.
- */
-public class GridHadoopV2Context extends JobContextImpl implements MapContext, ReduceContext {
-    /** Input reader to overriding of GridHadoopTaskContext input. */
-    private RecordReader reader;
-
-    /** Output writer to overriding of GridHadoopTaskContext output. */
-    private RecordWriter writer;
-
-    /** Output is provided by executor environment. */
-    private final GridHadoopTaskOutput output;
-
-    /** Input is provided by executor environment. */
-    private final GridHadoopTaskInput input;
-
-    /** Unique identifier for a task attempt. */
-    private final TaskAttemptID taskAttemptID;
-
-    /** Indicates that this task is to be cancelled. */
-    private volatile boolean cancelled;
-
-    /** Input split. */
-    private InputSplit inputSplit;
-
-    /** */
-    private final GridHadoopTaskContext ctx;
-
-    /** */
-    private String status;
-
-    /**
-     * @param ctx Context for IO operations.
-     */
-    public GridHadoopV2Context(GridHadoopV2TaskContext ctx) {
-        super(ctx.jobConf(), ctx.jobContext().getJobID());
-
-        taskAttemptID = ctx.attemptId();
-
-        conf.set("mapreduce.job.id", taskAttemptID.getJobID().toString());
-        conf.set("mapreduce.task.id", taskAttemptID.getTaskID().toString());
-
-        output = ctx.output();
-        input = ctx.input();
-
-        this.ctx = ctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public InputSplit getInputSplit() {
-        if (inputSplit == null) {
-            GridHadoopInputSplit split = ctx.taskInfo().inputSplit();
-
-            if (split == null)
-                return null;
-
-            if (split instanceof GridHadoopFileBlock) {
-                GridHadoopFileBlock fileBlock = (GridHadoopFileBlock)split;
-
-                inputSplit = new FileSplit(new Path(fileBlock.file()), fileBlock.start(), fileBlock.length(), null);
-            }
-            else if (split instanceof GridHadoopExternalSplit)
-                throw new UnsupportedOperationException(); // TODO
-            else if (split instanceof GridHadoopSplitWrapper)
-                inputSplit = (InputSplit) HadoopUtils.unwrapSplit((GridHadoopSplitWrapper) split);
-            else
-                throw new IllegalStateException();
-        }
-
-        return inputSplit;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean nextKeyValue() throws IOException, InterruptedException {
-        if (cancelled)
-            throw new HadoopTaskCancelledException("Task cancelled.");
-
-        return reader.nextKeyValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object getCurrentKey() throws IOException, InterruptedException {
-        if (reader != null)
-            return reader.getCurrentKey();
-
-        return input.key();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object getCurrentValue() throws IOException, InterruptedException {
-        return reader.getCurrentValue();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void write(Object key, Object val) throws IOException, InterruptedException {
-        if (cancelled)
-            throw new HadoopTaskCancelledException("Task cancelled.");
-
-        if (writer != null)
-            writer.write(key, val);
-        else {
-            try {
-                output.write(key, val);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IOException(e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public OutputCommitter getOutputCommitter() {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public TaskAttemptID getTaskAttemptID() {
-        return taskAttemptID;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setStatus(String msg) {
-        status = msg;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getStatus() {
-        return status;
-    }
-
-    /** {@inheritDoc} */
-    @Override public float getProgress() {
-        return 0.5f; // TODO
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counter getCounter(Enum<?> cntrName) {
-        return getCounter(cntrName.getDeclaringClass().getName(), cntrName.name());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counter getCounter(String grpName, String cntrName) {
-        return new GridHadoopV2Counter(ctx.counter(grpName, cntrName, HadoopLongCounter.class));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void progress() {
-        // No-op.
-    }
-
-    /**
-     * Overrides default input data reader.
-     *
-     * @param reader New reader.
-     */
-    public void reader(RecordReader reader) {
-        this.reader = reader;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean nextKey() throws IOException, InterruptedException {
-        if (cancelled)
-            throw new HadoopTaskCancelledException("Task cancelled.");
-
-        return input.next();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Iterable getValues() throws IOException, InterruptedException {
-        return new Iterable() {
-            @Override public Iterator iterator() {
-                return input.values();
-            }
-        };
-    }
-
-    /**
-     * @return Overridden output data writer.
-     */
-    public RecordWriter writer() {
-        return writer;
-    }
-
-    /**
-     * Overrides default output data writer.
-     *
-     * @param writer New writer.
-     */
-    public void writer(RecordWriter writer) {
-        this.writer = writer;
-    }
-
-    /**
-     * Cancels the task by stop the IO.
-     */
-    public void cancel() {
-        cancelled = true;
-    }
-}


[02/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
new file mode 100644
index 0000000..f896daa
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
@@ -0,0 +1,146 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+
+/**
+ * Task executor.
+ */
+public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
+    /** Job tracker. */
+    private HadoopJobTracker jobTracker;
+
+    /** */
+    private final ConcurrentMap<GridHadoopJobId, Collection<GridHadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
+
+    /** Executor service to run tasks. */
+    private GridHadoopExecutorService exec;
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        super.onKernalStart();
+
+        jobTracker = ctx.jobTracker();
+
+        exec = new GridHadoopExecutorService(log, ctx.kernalContext().gridName(),
+            ctx.configuration().getMaxParallelTasks(), ctx.configuration().getMaxTaskQueueSize());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        if (exec != null) {
+            exec.shutdown(3000);
+
+            if (cancel) {
+                for (GridHadoopJobId jobId : jobs.keySet())
+                    cancelTasks(jobId);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) {
+        if (exec != null && !exec.shutdown(30000))
+            U.warn(log, "Failed to finish running tasks in 30 sec.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void run(final GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException {
+        if (log.isDebugEnabled())
+            log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() +
+                ", tasksCnt=" + tasks.size() + ']');
+
+        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(job.id());
+
+        if (executedTasks == null) {
+            executedTasks = new GridConcurrentHashSet<>();
+
+            Collection<GridHadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks);
+
+            assert extractedCol == null;
+        }
+
+        final Collection<GridHadoopRunnableTask> finalExecutedTasks = executedTasks;
+
+        for (final GridHadoopTaskInfo info : tasks) {
+            assert info != null;
+
+            GridHadoopRunnableTask task = new GridHadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
+                ctx.localNodeId()) {
+                @Override protected void onTaskFinished(GridHadoopTaskStatus status) {
+                    if (log.isDebugEnabled())
+                        log.debug("Finished task execution [jobId=" + job.id() + ", taskInfo=" + info + ", " +
+                            "waitTime=" + waitTime() + ", execTime=" + executionTime() + ']');
+
+                    finalExecutedTasks.remove(this);
+
+                    jobTracker.onTaskFinished(info, status);
+                }
+
+                @Override protected GridHadoopTaskInput createInput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+                    return ctx.shuffle().input(taskCtx);
+                }
+
+                @Override protected GridHadoopTaskOutput createOutput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+                    return ctx.shuffle().output(taskCtx);
+                }
+            };
+
+            executedTasks.add(task);
+
+            exec.submit(task);
+        }
+    }
+
+    /**
+     * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks
+     * for this job ID.
+     * <p>
+     * It is guaranteed that this method will not be called concurrently with
+     * {@link #run(GridHadoopJob, Collection)} method. No more job submissions will be performed via
+     * {@link #run(GridHadoopJob, Collection)} method for given job ID after this method is called.
+     *
+     * @param jobId Job ID to cancel.
+     */
+    @Override public void cancelTasks(GridHadoopJobId jobId) {
+        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(jobId);
+
+        if (executedTasks != null) {
+            for (GridHadoopRunnableTask task : executedTasks)
+                task.cancel();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException {
+        if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
+            Collection<GridHadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
+
+            assert executedTasks == null || executedTasks.isEmpty();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
new file mode 100644
index 0000000..a3d3bf7
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
@@ -0,0 +1,57 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+
+import java.util.*;
+
+/**
+ * Common superclass for task executor.
+ */
+public abstract class HadoopTaskExecutorAdapter extends HadoopComponent {
+    /**
+     * Runs tasks.
+     *
+     * @param job Job.
+     * @param tasks Tasks.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract void run(final GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException;
+
+    /**
+     * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks
+     * for this job ID.
+     * <p>
+     * It is guaranteed that this method will not be called concurrently with
+     * {@link #run(GridHadoopJob, Collection)} method. No more job submissions will be performed via
+     * {@link #run(GridHadoopJob, Collection)} method for given job ID after this method is called.
+     *
+     * @param jobId Job ID to cancel.
+     */
+    public abstract void cancelTasks(GridHadoopJobId jobId) throws IgniteCheckedException;
+
+    /**
+     * On job state change callback;
+     *
+     * @param meta Job metadata.
+     */
+    public abstract void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
deleted file mode 100644
index 72185c0..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
+++ /dev/null
@@ -1,960 +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.taskexecutor.external;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.locks.*;
-
-import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
-
-/**
- * External process registry. Handles external process lifecycle.
- */
-public class GridHadoopExternalTaskExecutor extends GridHadoopTaskExecutorAdapter {
-    /** Hadoop context. */
-    private GridHadoopContext ctx;
-
-    /** */
-    private String javaCmd;
-
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Node process descriptor. */
-    private GridHadoopProcessDescriptor nodeDesc;
-
-    /** Output base. */
-    private File outputBase;
-
-    /** Path separator. */
-    private String pathSep;
-
-    /** Hadoop external communication. */
-    private GridHadoopExternalCommunication comm;
-
-    /** Starting processes. */
-    private final ConcurrentMap<UUID, HadoopProcess> runningProcsByProcId = new ConcurrentHashMap8<>();
-
-    /** Starting processes. */
-    private final ConcurrentMap<GridHadoopJobId, HadoopProcess> runningProcsByJobId = new ConcurrentHashMap8<>();
-
-    /** Busy lock. */
-    private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock();
-
-    /** Job tracker. */
-    private GridHadoopJobTracker jobTracker;
-
-    /** {@inheritDoc} */
-    @Override public void start(GridHadoopContext ctx) throws IgniteCheckedException {
-        this.ctx = ctx;
-
-        log = ctx.kernalContext().log(GridHadoopExternalTaskExecutor.class);
-
-        outputBase = U.resolveWorkDirectory("hadoop", false);
-
-        pathSep = System.getProperty("path.separator", U.isWindows() ? ";" : ":");
-
-        initJavaCommand();
-
-        comm = new GridHadoopExternalCommunication(
-            ctx.localNodeId(),
-            UUID.randomUUID(),
-            ctx.kernalContext().config().getMarshaller(),
-            log,
-            ctx.kernalContext().getSystemExecutorService(),
-            ctx.kernalContext().gridName());
-
-        comm.setListener(new MessageListener());
-
-        comm.start();
-
-        nodeDesc = comm.localProcessDescriptor();
-
-        ctx.kernalContext().ports().registerPort(nodeDesc.tcpPort(), IgnitePortProtocol.TCP,
-            GridHadoopExternalTaskExecutor.class);
-
-        if (nodeDesc.sharedMemoryPort() != -1)
-            ctx.kernalContext().ports().registerPort(nodeDesc.sharedMemoryPort(), IgnitePortProtocol.TCP,
-                GridHadoopExternalTaskExecutor.class);
-
-        jobTracker = ctx.jobTracker();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) {
-        busyLock.writeLock();
-
-        try {
-            comm.stop();
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to gracefully stop external hadoop communication server (will shutdown anyway)", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onJobStateChanged(final GridHadoopJobMetadata meta) {
-        final HadoopProcess proc = runningProcsByJobId.get(meta.jobId());
-
-        // If we have a local process for this job.
-        if (proc != null) {
-            if (log.isDebugEnabled())
-                log.debug("Updating job information for remote task process [proc=" + proc + ", meta=" + meta + ']');
-
-            if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
-                if (log.isDebugEnabled())
-                    log.debug("Completed job execution, will terminate child process [jobId=" + meta.jobId() +
-                        ", proc=" + proc + ']');
-
-                runningProcsByJobId.remove(meta.jobId());
-                runningProcsByProcId.remove(proc.descriptor().processId());
-
-                proc.terminate();
-
-                return;
-            }
-
-            if (proc.initFut.isDone()) {
-                if (!proc.initFut.isFailed())
-                    sendJobInfoUpdate(proc, meta);
-                else if (log.isDebugEnabled())
-                    log.debug("Failed to initialize child process (will skip job state notification) " +
-                        "[jobId=" + meta.jobId() + ", meta=" + meta + ']');
-            }
-            else {
-                proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-                    @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
-                        try {
-                            f.get();
-
-                            sendJobInfoUpdate(proc, meta);
-                        }
-                        catch (IgniteCheckedException e) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to initialize child process (will skip job state notification) " +
-                                    "[jobId=" + meta.jobId() + ", meta=" + meta + ", err=" + e + ']');
-                        }
-
-                    }
-                });
-            }
-        }
-        else if (ctx.isParticipating(meta)) {
-            GridHadoopJob job;
-
-            try {
-                job = jobTracker.job(meta.jobId(), meta.jobInfo());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to get job: " + meta.jobId(), e);
-
-                return;
-            }
-
-            startProcess(job, meta.mapReducePlan());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("ConstantConditions")
-    @Override public void run(final GridHadoopJob job, final Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock()) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to start hadoop tasks (grid is stopping, will ignore).");
-
-            return;
-        }
-
-        try {
-            HadoopProcess proc = runningProcsByJobId.get(job.id());
-
-            GridHadoopTaskType taskType = F.first(tasks).type();
-
-            if (taskType == GridHadoopTaskType.SETUP || taskType == GridHadoopTaskType.ABORT ||
-                taskType == GridHadoopTaskType.COMMIT) {
-                if (proc == null || proc.terminated()) {
-                    runningProcsByJobId.remove(job.id(), proc);
-
-                    // Start new process for ABORT task since previous processes were killed.
-                    proc = startProcess(job, jobTracker.plan(job.id()));
-
-                    if (log.isDebugEnabled())
-                        log.debug("Starting new process for maintenance task [jobId=" + job.id() +
-                            ", proc=" + proc + ", taskType=" + taskType + ']');
-                }
-            }
-            else
-                assert proc != null : "Missing started process for task execution request: " + job.id() +
-                    ", tasks=" + tasks;
-
-            final HadoopProcess proc0 = proc;
-
-            proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-                @Override public void apply(
-                    IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
-                    if (!busyLock.tryReadLock())
-                        return;
-
-                    try {
-                        f.get();
-
-                        proc0.addTasks(tasks);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Sending task execution request to child process [jobId=" + job.id() +
-                                ", proc=" + proc0 + ", tasks=" + tasks + ']');
-
-                        sendExecutionRequest(proc0, job, tasks);
-                    }
-                    catch (IgniteCheckedException e) {
-                        notifyTasksFailed(tasks, FAILED, e);
-                    }
-                    finally {
-                        busyLock.readUnlock();
-                    }
-                }
-            });
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cancelTasks(GridHadoopJobId jobId) {
-        HadoopProcess proc = runningProcsByJobId.get(jobId);
-
-        if (proc != null)
-            proc.terminate();
-    }
-
-    /**
-     * Sends execution request to remote node.
-     *
-     * @param proc Process to send request to.
-     * @param job Job instance.
-     * @param tasks Collection of tasks to execute in started process.
-     */
-    private void sendExecutionRequest(HadoopProcess proc, GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks)
-        throws IgniteCheckedException {
-        // Must synchronize since concurrent process crash may happen and will receive onConnectionLost().
-        proc.lock();
-
-        try {
-            if (proc.terminated()) {
-                notifyTasksFailed(tasks, CRASHED, null);
-
-                return;
-            }
-
-            GridHadoopTaskExecutionRequest req = new GridHadoopTaskExecutionRequest();
-
-            req.jobId(job.id());
-            req.jobInfo(job.info());
-            req.tasks(tasks);
-
-            comm.sendMessage(proc.descriptor(), req);
-        }
-        finally {
-            proc.unlock();
-        }
-    }
-
-    /**
-     * @return External task metadata.
-     */
-    private GridHadoopExternalTaskMetadata buildTaskMeta() {
-        GridHadoopExternalTaskMetadata meta = new GridHadoopExternalTaskMetadata();
-
-        meta.classpath(Arrays.asList(System.getProperty("java.class.path").split(File.pathSeparator)));
-        meta.jvmOptions(Arrays.asList("-Xmx1g", "-ea", "-XX:+UseConcMarkSweepGC", "-XX:+CMSClassUnloadingEnabled",
-            "-DIGNITE_HOME=" + U.getIgniteHome()));
-
-        return meta;
-    }
-
-    /**
-     * @param tasks Tasks to notify about.
-     * @param state Fail state.
-     * @param e Optional error.
-     */
-    private void notifyTasksFailed(Iterable<GridHadoopTaskInfo> tasks, GridHadoopTaskState state, Throwable e) {
-        GridHadoopTaskStatus fail = new GridHadoopTaskStatus(state, e);
-
-        for (GridHadoopTaskInfo task : tasks)
-            jobTracker.onTaskFinished(task, fail);
-    }
-
-    /**
-     * Starts process template that will be ready to execute Hadoop tasks.
-     *
-     * @param job Job instance.
-     * @param plan Map reduce plan.
-     */
-    private HadoopProcess startProcess(final GridHadoopJob job, final GridHadoopMapReducePlan plan) {
-        final UUID childProcId = UUID.randomUUID();
-
-        GridHadoopJobId jobId = job.id();
-
-        final GridHadoopProcessFuture fut = new GridHadoopProcessFuture(childProcId, jobId, ctx.kernalContext());
-
-        final HadoopProcess proc = new HadoopProcess(jobId, fut, plan.reducers(ctx.localNodeId()));
-
-        HadoopProcess old = runningProcsByJobId.put(jobId, proc);
-
-        assert old == null;
-
-        old = runningProcsByProcId.put(childProcId, proc);
-
-        assert old == null;
-
-        ctx.kernalContext().closure().runLocalSafe(new Runnable() {
-            @Override public void run() {
-                if (!busyLock.tryReadLock()) {
-                    fut.onDone(new IgniteCheckedException("Failed to start external process (grid is stopping)."));
-
-                    return;
-                }
-
-                try {
-                    GridHadoopExternalTaskMetadata startMeta = buildTaskMeta();
-
-                    if (log.isDebugEnabled())
-                        log.debug("Created hadoop child process metadata for job [job=" + job +
-                            ", childProcId=" + childProcId + ", taskMeta=" + startMeta + ']');
-
-                    Process proc = startJavaProcess(childProcId, startMeta, job);
-
-                    BufferedReader rdr = new BufferedReader(new InputStreamReader(proc.getInputStream()));
-
-                    String line;
-
-                    // Read up all the process output.
-                    while ((line = rdr.readLine()) != null) {
-                        if (log.isDebugEnabled())
-                            log.debug("Tracing process output: " + line);
-
-                        if ("Started".equals(line)) {
-                            // Process started successfully, it should not write anything more to the output stream.
-                            if (log.isDebugEnabled())
-                                log.debug("Successfully started child process [childProcId=" + childProcId +
-                                    ", meta=" + job + ']');
-
-                            fut.onProcessStarted(proc);
-
-                            break;
-                        }
-                        else if ("Failed".equals(line)) {
-                            StringBuilder sb = new StringBuilder("Failed to start child process: " + job + "\n");
-
-                            while ((line = rdr.readLine()) != null)
-                                sb.append("    ").append(line).append("\n");
-
-                            // Cut last character.
-                            sb.setLength(sb.length() - 1);
-
-                            log.warning(sb.toString());
-
-                            fut.onDone(new IgniteCheckedException(sb.toString()));
-
-                            break;
-                        }
-                    }
-                }
-                catch (Throwable e) {
-                    fut.onDone(new IgniteCheckedException("Failed to initialize child process: " + job, e));
-                }
-                finally {
-                    busyLock.readUnlock();
-                }
-            }
-        }, true);
-
-        fut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-            @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
-                try {
-                    // Make sure there were no exceptions.
-                    f.get();
-
-                    prepareForJob(proc, job, plan);
-                }
-                catch (IgniteCheckedException ignore) {
-                    // Exception is printed in future's onDone() method.
-                }
-            }
-        });
-
-        return proc;
-    }
-
-    /**
-     * Checks that java local command is available.
-     *
-     * @throws IgniteCheckedException If initialization failed.
-     */
-    private void initJavaCommand() throws IgniteCheckedException {
-        String javaHome = System.getProperty("java.home");
-
-        if (javaHome == null)
-            javaHome = System.getenv("JAVA_HOME");
-
-        if (javaHome == null)
-            throw new IgniteCheckedException("Failed to locate JAVA_HOME.");
-
-        javaCmd = javaHome + File.separator + "bin" + File.separator + (U.isWindows() ? "java.exe" : "java");
-
-        try {
-            Process proc = new ProcessBuilder(javaCmd, "-version").redirectErrorStream(true).start();
-
-            Collection<String> out = readProcessOutput(proc);
-
-            int res = proc.waitFor();
-
-            if (res != 0)
-                throw new IgniteCheckedException("Failed to execute 'java -version' command (process finished with nonzero " +
-                    "code) [exitCode=" + res + ", javaCmd='" + javaCmd + "', msg=" + F.first(out) + ']');
-
-            if (log.isInfoEnabled()) {
-                log.info("Will use java for external task execution: ");
-
-                for (String s : out)
-                    log.info("    " + s);
-            }
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to check java for external task execution.", e);
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteCheckedException("Failed to wait for process completion (thread got interrupted).", e);
-        }
-    }
-
-    /**
-     * Reads process output line-by-line.
-     *
-     * @param proc Process to read output.
-     * @return Read lines.
-     * @throws IOException If read failed.
-     */
-    private Collection<String> readProcessOutput(Process proc) throws IOException {
-        BufferedReader rdr = new BufferedReader(new InputStreamReader(proc.getInputStream()));
-
-        Collection<String> res = new ArrayList<>();
-
-        String s;
-
-        while ((s = rdr.readLine()) != null)
-            res.add(s);
-
-        return res;
-    }
-
-    /**
-     * Builds process from metadata.
-     *
-     * @param childProcId Child process ID.
-     * @param startMeta Metadata.
-     * @param job Job.
-     * @return Started process.
-     */
-    private Process startJavaProcess(UUID childProcId, GridHadoopExternalTaskMetadata startMeta,
-        GridHadoopJob job) throws Exception {
-        String outFldr = jobWorkFolder(job.id()) + File.separator + childProcId;
-
-        if (log.isDebugEnabled())
-            log.debug("Will write process log output to: " + outFldr);
-
-        List<String> cmd = new ArrayList<>();
-
-        File workDir = U.resolveWorkDirectory("", false);
-
-        cmd.add(javaCmd);
-        cmd.addAll(startMeta.jvmOptions());
-        cmd.add("-cp");
-        cmd.add(buildClasspath(startMeta.classpath()));
-        cmd.add(GridHadoopExternalProcessStarter.class.getName());
-        cmd.add("-cpid");
-        cmd.add(String.valueOf(childProcId));
-        cmd.add("-ppid");
-        cmd.add(String.valueOf(nodeDesc.processId()));
-        cmd.add("-nid");
-        cmd.add(String.valueOf(nodeDesc.parentNodeId()));
-        cmd.add("-addr");
-        cmd.add(nodeDesc.address());
-        cmd.add("-tport");
-        cmd.add(String.valueOf(nodeDesc.tcpPort()));
-        cmd.add("-sport");
-        cmd.add(String.valueOf(nodeDesc.sharedMemoryPort()));
-        cmd.add("-out");
-        cmd.add(outFldr);
-        cmd.add("-wd");
-        cmd.add(workDir.getAbsolutePath());
-
-        return new ProcessBuilder(cmd)
-            .redirectErrorStream(true)
-            .directory(workDir)
-            .start();
-    }
-
-    /**
-     * Gets job work folder.
-     *
-     * @param jobId Job ID.
-     * @return Job work folder.
-     */
-    private String jobWorkFolder(GridHadoopJobId jobId) {
-        return outputBase + File.separator + "Job_" + jobId;
-    }
-
-    /**
-     * @param cp Classpath collection.
-     * @return Classpath string.
-     */
-    private String buildClasspath(Collection<String> cp) {
-        assert !cp.isEmpty();
-
-        StringBuilder sb = new StringBuilder();
-
-        for (String s : cp)
-            sb.append(s).append(pathSep);
-
-        sb.setLength(sb.length() - 1);
-
-        return sb.toString();
-    }
-
-    /**
-     * Sends job info update request to remote process.
-     *
-     * @param proc Process to send request to.
-     * @param meta Job metadata.
-     */
-    private void sendJobInfoUpdate(HadoopProcess proc, GridHadoopJobMetadata meta) {
-        Map<Integer, GridHadoopProcessDescriptor> rdcAddrs = meta.reducersAddresses();
-
-        int rdcNum = meta.mapReducePlan().reducers();
-
-        GridHadoopProcessDescriptor[] addrs = null;
-
-        if (rdcAddrs != null && rdcAddrs.size() == rdcNum) {
-            addrs = new GridHadoopProcessDescriptor[rdcNum];
-
-            for (int i = 0; i < rdcNum; i++) {
-                GridHadoopProcessDescriptor desc = rdcAddrs.get(i);
-
-                assert desc != null : "Missing reducing address [meta=" + meta + ", rdc=" + i + ']';
-
-                addrs[i] = desc;
-            }
-        }
-
-        try {
-            comm.sendMessage(proc.descriptor(), new GridHadoopJobInfoUpdateRequest(proc.jobId, meta.phase(), addrs));
-        }
-        catch (IgniteCheckedException e) {
-            if (!proc.terminated()) {
-                log.error("Failed to send job state update message to remote child process (will kill the process) " +
-                    "[jobId=" + proc.jobId + ", meta=" + meta + ']', e);
-
-                proc.terminate();
-            }
-        }
-    }
-
-    /**
-     * Sends prepare request to remote process.
-     *
-     * @param proc Process to send request to.
-     * @param job Job.
-     * @param plan Map reduce plan.
-     */
-    private void prepareForJob(HadoopProcess proc, GridHadoopJob job, GridHadoopMapReducePlan plan) {
-        try {
-            comm.sendMessage(proc.descriptor(), new GridHadoopPrepareForJobRequest(job.id(), job.info(),
-                plan.reducers(), plan.reducers(ctx.localNodeId())));
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send job prepare request to remote process [proc=" + proc + ", job=" + job +
-                ", plan=" + plan + ']', e);
-
-            proc.terminate();
-        }
-    }
-
-    /**
-     * Processes task finished message.
-     *
-     * @param desc Remote process descriptor.
-     * @param taskMsg Task finished message.
-     */
-    private void processTaskFinishedMessage(GridHadoopProcessDescriptor desc, GridHadoopTaskFinishedMessage taskMsg) {
-        HadoopProcess proc = runningProcsByProcId.get(desc.processId());
-
-        if (proc != null)
-            proc.removeTask(taskMsg.taskInfo());
-
-        jobTracker.onTaskFinished(taskMsg.taskInfo(), taskMsg.status());
-    }
-
-    /**
-     *
-     */
-    private class MessageListener implements GridHadoopMessageListener {
-        /** {@inheritDoc} */
-        @Override public void onMessageReceived(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
-            if (!busyLock.tryReadLock())
-                return;
-
-            try {
-                if (msg instanceof GridHadoopProcessStartedAck) {
-                    HadoopProcess proc = runningProcsByProcId.get(desc.processId());
-
-                    assert proc != null : "Missing child process for processId: " + desc;
-
-                    GridHadoopProcessFuture fut = proc.initFut;
-
-                    if (fut != null)
-                        fut.onReplyReceived(desc);
-                    // Safety.
-                    else
-                        log.warning("Failed to find process start future (will ignore): " + desc);
-                }
-                else if (msg instanceof GridHadoopTaskFinishedMessage) {
-                    GridHadoopTaskFinishedMessage taskMsg = (GridHadoopTaskFinishedMessage)msg;
-
-                    processTaskFinishedMessage(desc, taskMsg);
-                }
-                else
-                    log.warning("Unexpected message received by node [desc=" + desc + ", msg=" + msg + ']');
-            }
-            finally {
-                busyLock.readUnlock();
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onConnectionLost(GridHadoopProcessDescriptor desc) {
-            if (!busyLock.tryReadLock())
-                return;
-
-            try {
-                if (desc == null) {
-                    U.warn(log, "Handshake failed.");
-
-                    return;
-                }
-
-                // Notify job tracker about failed tasks.
-                HadoopProcess proc = runningProcsByProcId.get(desc.processId());
-
-                if (proc != null) {
-                    Collection<GridHadoopTaskInfo> tasks = proc.tasks();
-
-                    if (!F.isEmpty(tasks)) {
-                        log.warning("Lost connection with alive process (will terminate): " + desc);
-
-                        GridHadoopTaskStatus status = new GridHadoopTaskStatus(CRASHED,
-                            new IgniteCheckedException("Failed to run tasks (external process finished unexpectedly): " + desc));
-
-                        for (GridHadoopTaskInfo info : tasks)
-                            jobTracker.onTaskFinished(info, status);
-
-                        runningProcsByJobId.remove(proc.jobId(), proc);
-                    }
-
-                    // Safety.
-                    proc.terminate();
-                }
-            }
-            finally {
-                busyLock.readUnlock();
-            }
-        }
-    }
-
-    /**
-     * Hadoop process.
-     */
-    private static class HadoopProcess extends ReentrantLock {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Job ID. */
-        private final GridHadoopJobId jobId;
-
-        /** Process. */
-        private Process proc;
-
-        /** Init future. Completes when process is ready to receive messages. */
-        private final GridHadoopProcessFuture initFut;
-
-        /** Process descriptor. */
-        private GridHadoopProcessDescriptor procDesc;
-
-        /** Reducers planned for this process. */
-        private Collection<Integer> reducers;
-
-        /** Tasks. */
-        private final Collection<GridHadoopTaskInfo> tasks = new ConcurrentLinkedDeque8<>();
-
-        /** Terminated flag. */
-        private volatile boolean terminated;
-
-        /**
-         * @param jobId Job ID.
-         * @param initFut Init future.
-         */
-        private HadoopProcess(GridHadoopJobId jobId, GridHadoopProcessFuture initFut,
-            int[] reducers) {
-            this.jobId = jobId;
-            this.initFut = initFut;
-
-            if (!F.isEmpty(reducers)) {
-                this.reducers = new ArrayList<>(reducers.length);
-
-                for (int r : reducers)
-                    this.reducers.add(r);
-            }
-        }
-
-        /**
-         * @return Communication process descriptor.
-         */
-        private GridHadoopProcessDescriptor descriptor() {
-            return procDesc;
-        }
-
-        /**
-         * @return Job ID.
-         */
-        public GridHadoopJobId jobId() {
-            return jobId;
-        }
-
-        /**
-         * Initialized callback.
-         *
-         * @param proc Java process representation.
-         * @param procDesc Process descriptor.
-         */
-        private void onInitialized(Process proc, GridHadoopProcessDescriptor procDesc) {
-            this.proc = proc;
-            this.procDesc = procDesc;
-        }
-
-        /**
-         * Terminates process (kills it).
-         */
-        private void terminate() {
-            // Guard against concurrent message sending.
-            lock();
-
-            try {
-                terminated = true;
-
-                if (!initFut.isDone())
-                    initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-                        @Override public void apply(
-                            IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
-                            proc.destroy();
-                        }
-                    });
-                else
-                    proc.destroy();
-            }
-            finally {
-                unlock();
-            }
-        }
-
-        /**
-         * @return Terminated flag.
-         */
-        private boolean terminated() {
-            return terminated;
-        }
-
-        /**
-         * Sets process tasks.
-         *
-         * @param tasks Tasks to set.
-         */
-        private void addTasks(Collection<GridHadoopTaskInfo> tasks) {
-            this.tasks.addAll(tasks);
-        }
-
-        /**
-         * Removes task when it was completed.
-         *
-         * @param task Task to remove.
-         */
-        private void removeTask(GridHadoopTaskInfo task) {
-            if (tasks != null)
-                tasks.remove(task);
-        }
-
-        /**
-         * @return Collection of tasks.
-         */
-        private Collection<GridHadoopTaskInfo> tasks() {
-            return tasks;
-        }
-
-        /**
-         * @return Planned reducers.
-         */
-        private Collection<Integer> reducers() {
-            return reducers;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(HadoopProcess.class, this);
-        }
-    }
-
-    /**
-     *
-     */
-    private class GridHadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Child process ID. */
-        private UUID childProcId;
-
-        /** Job ID. */
-        private GridHadoopJobId jobId;
-
-        /** Process descriptor. */
-        private GridHadoopProcessDescriptor desc;
-
-        /** Running process. */
-        private Process proc;
-
-        /** Process started flag. */
-        private volatile boolean procStarted;
-
-        /** Reply received flag. */
-        private volatile boolean replyReceived;
-
-        /** Logger. */
-        private final IgniteLogger log = GridHadoopExternalTaskExecutor.this.log;
-
-        /**
-         * Empty constructor.
-         */
-        public GridHadoopProcessFuture() {
-            // No-op.
-        }
-
-        /**
-         * @param ctx Kernal context.
-         */
-        private GridHadoopProcessFuture(UUID childProcId, GridHadoopJobId jobId, GridKernalContext ctx) {
-            super(ctx);
-
-            this.childProcId = childProcId;
-            this.jobId = jobId;
-        }
-
-        /**
-         * Process started callback.
-         */
-        public void onProcessStarted(Process proc) {
-            this.proc = proc;
-
-            procStarted = true;
-
-            if (procStarted && replyReceived)
-                onDone(F.t(proc, desc));
-        }
-
-        /**
-         * Reply received callback.
-         */
-        public void onReplyReceived(GridHadoopProcessDescriptor desc) {
-            assert childProcId.equals(desc.processId());
-
-            this.desc = desc;
-
-            replyReceived = true;
-
-            if (procStarted && replyReceived)
-                onDone(F.t(proc, desc));
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable IgniteBiTuple<Process, GridHadoopProcessDescriptor> res,
-            @Nullable Throwable err) {
-            if (err == null) {
-                HadoopProcess proc = runningProcsByProcId.get(childProcId);
-
-                assert proc != null;
-
-                assert proc.initFut == this;
-
-                proc.onInitialized(res.get1(), res.get2());
-
-                if (!F.isEmpty(proc.reducers()))
-                    jobTracker.onExternalMappersInitialized(jobId, proc.reducers(), desc);
-            }
-            else {
-                // Clean up since init failed.
-                runningProcsByJobId.remove(jobId);
-                runningProcsByProcId.remove(childProcId);
-            }
-
-            if (super.onDone(res, err)) {
-                if (err == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Initialized child process for external task execution [jobId=" + jobId +
-                            ", desc=" + desc + ", initTime=" + duration() + ']');
-                }
-                else
-                    U.error(log, "Failed to initialize child process for external task execution [jobId=" + jobId +
-                        ", desc=" + desc + ']', err);
-
-                return true;
-            }
-
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
new file mode 100644
index 0000000..616d383
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
@@ -0,0 +1,960 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
+
+/**
+ * External process registry. Handles external process lifecycle.
+ */
+public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
+    /** Hadoop context. */
+    private HadoopContext ctx;
+
+    /** */
+    private String javaCmd;
+
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** Node process descriptor. */
+    private GridHadoopProcessDescriptor nodeDesc;
+
+    /** Output base. */
+    private File outputBase;
+
+    /** Path separator. */
+    private String pathSep;
+
+    /** Hadoop external communication. */
+    private GridHadoopExternalCommunication comm;
+
+    /** Starting processes. */
+    private final ConcurrentMap<UUID, HadoopProcess> runningProcsByProcId = new ConcurrentHashMap8<>();
+
+    /** Starting processes. */
+    private final ConcurrentMap<GridHadoopJobId, HadoopProcess> runningProcsByJobId = new ConcurrentHashMap8<>();
+
+    /** Busy lock. */
+    private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock();
+
+    /** Job tracker. */
+    private HadoopJobTracker jobTracker;
+
+    /** {@inheritDoc} */
+    @Override public void start(HadoopContext ctx) throws IgniteCheckedException {
+        this.ctx = ctx;
+
+        log = ctx.kernalContext().log(HadoopExternalTaskExecutor.class);
+
+        outputBase = U.resolveWorkDirectory("hadoop", false);
+
+        pathSep = System.getProperty("path.separator", U.isWindows() ? ";" : ":");
+
+        initJavaCommand();
+
+        comm = new GridHadoopExternalCommunication(
+            ctx.localNodeId(),
+            UUID.randomUUID(),
+            ctx.kernalContext().config().getMarshaller(),
+            log,
+            ctx.kernalContext().getSystemExecutorService(),
+            ctx.kernalContext().gridName());
+
+        comm.setListener(new MessageListener());
+
+        comm.start();
+
+        nodeDesc = comm.localProcessDescriptor();
+
+        ctx.kernalContext().ports().registerPort(nodeDesc.tcpPort(), IgnitePortProtocol.TCP,
+            HadoopExternalTaskExecutor.class);
+
+        if (nodeDesc.sharedMemoryPort() != -1)
+            ctx.kernalContext().ports().registerPort(nodeDesc.sharedMemoryPort(), IgnitePortProtocol.TCP,
+                HadoopExternalTaskExecutor.class);
+
+        jobTracker = ctx.jobTracker();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) {
+        busyLock.writeLock();
+
+        try {
+            comm.stop();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to gracefully stop external hadoop communication server (will shutdown anyway)", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onJobStateChanged(final GridHadoopJobMetadata meta) {
+        final HadoopProcess proc = runningProcsByJobId.get(meta.jobId());
+
+        // If we have a local process for this job.
+        if (proc != null) {
+            if (log.isDebugEnabled())
+                log.debug("Updating job information for remote task process [proc=" + proc + ", meta=" + meta + ']');
+
+            if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
+                if (log.isDebugEnabled())
+                    log.debug("Completed job execution, will terminate child process [jobId=" + meta.jobId() +
+                        ", proc=" + proc + ']');
+
+                runningProcsByJobId.remove(meta.jobId());
+                runningProcsByProcId.remove(proc.descriptor().processId());
+
+                proc.terminate();
+
+                return;
+            }
+
+            if (proc.initFut.isDone()) {
+                if (!proc.initFut.isFailed())
+                    sendJobInfoUpdate(proc, meta);
+                else if (log.isDebugEnabled())
+                    log.debug("Failed to initialize child process (will skip job state notification) " +
+                        "[jobId=" + meta.jobId() + ", meta=" + meta + ']');
+            }
+            else {
+                proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                    @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                        try {
+                            f.get();
+
+                            sendJobInfoUpdate(proc, meta);
+                        }
+                        catch (IgniteCheckedException e) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to initialize child process (will skip job state notification) " +
+                                    "[jobId=" + meta.jobId() + ", meta=" + meta + ", err=" + e + ']');
+                        }
+
+                    }
+                });
+            }
+        }
+        else if (ctx.isParticipating(meta)) {
+            GridHadoopJob job;
+
+            try {
+                job = jobTracker.job(meta.jobId(), meta.jobInfo());
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to get job: " + meta.jobId(), e);
+
+                return;
+            }
+
+            startProcess(job, meta.mapReducePlan());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("ConstantConditions")
+    @Override public void run(final GridHadoopJob job, final Collection<GridHadoopTaskInfo> tasks) throws IgniteCheckedException {
+        if (!busyLock.tryReadLock()) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to start hadoop tasks (grid is stopping, will ignore).");
+
+            return;
+        }
+
+        try {
+            HadoopProcess proc = runningProcsByJobId.get(job.id());
+
+            GridHadoopTaskType taskType = F.first(tasks).type();
+
+            if (taskType == GridHadoopTaskType.SETUP || taskType == GridHadoopTaskType.ABORT ||
+                taskType == GridHadoopTaskType.COMMIT) {
+                if (proc == null || proc.terminated()) {
+                    runningProcsByJobId.remove(job.id(), proc);
+
+                    // Start new process for ABORT task since previous processes were killed.
+                    proc = startProcess(job, jobTracker.plan(job.id()));
+
+                    if (log.isDebugEnabled())
+                        log.debug("Starting new process for maintenance task [jobId=" + job.id() +
+                            ", proc=" + proc + ", taskType=" + taskType + ']');
+                }
+            }
+            else
+                assert proc != null : "Missing started process for task execution request: " + job.id() +
+                    ", tasks=" + tasks;
+
+            final HadoopProcess proc0 = proc;
+
+            proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                @Override public void apply(
+                    IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                    if (!busyLock.tryReadLock())
+                        return;
+
+                    try {
+                        f.get();
+
+                        proc0.addTasks(tasks);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Sending task execution request to child process [jobId=" + job.id() +
+                                ", proc=" + proc0 + ", tasks=" + tasks + ']');
+
+                        sendExecutionRequest(proc0, job, tasks);
+                    }
+                    catch (IgniteCheckedException e) {
+                        notifyTasksFailed(tasks, FAILED, e);
+                    }
+                    finally {
+                        busyLock.readUnlock();
+                    }
+                }
+            });
+        }
+        finally {
+            busyLock.readUnlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelTasks(GridHadoopJobId jobId) {
+        HadoopProcess proc = runningProcsByJobId.get(jobId);
+
+        if (proc != null)
+            proc.terminate();
+    }
+
+    /**
+     * Sends execution request to remote node.
+     *
+     * @param proc Process to send request to.
+     * @param job Job instance.
+     * @param tasks Collection of tasks to execute in started process.
+     */
+    private void sendExecutionRequest(HadoopProcess proc, GridHadoopJob job, Collection<GridHadoopTaskInfo> tasks)
+        throws IgniteCheckedException {
+        // Must synchronize since concurrent process crash may happen and will receive onConnectionLost().
+        proc.lock();
+
+        try {
+            if (proc.terminated()) {
+                notifyTasksFailed(tasks, CRASHED, null);
+
+                return;
+            }
+
+            GridHadoopTaskExecutionRequest req = new GridHadoopTaskExecutionRequest();
+
+            req.jobId(job.id());
+            req.jobInfo(job.info());
+            req.tasks(tasks);
+
+            comm.sendMessage(proc.descriptor(), req);
+        }
+        finally {
+            proc.unlock();
+        }
+    }
+
+    /**
+     * @return External task metadata.
+     */
+    private GridHadoopExternalTaskMetadata buildTaskMeta() {
+        GridHadoopExternalTaskMetadata meta = new GridHadoopExternalTaskMetadata();
+
+        meta.classpath(Arrays.asList(System.getProperty("java.class.path").split(File.pathSeparator)));
+        meta.jvmOptions(Arrays.asList("-Xmx1g", "-ea", "-XX:+UseConcMarkSweepGC", "-XX:+CMSClassUnloadingEnabled",
+            "-DIGNITE_HOME=" + U.getIgniteHome()));
+
+        return meta;
+    }
+
+    /**
+     * @param tasks Tasks to notify about.
+     * @param state Fail state.
+     * @param e Optional error.
+     */
+    private void notifyTasksFailed(Iterable<GridHadoopTaskInfo> tasks, GridHadoopTaskState state, Throwable e) {
+        GridHadoopTaskStatus fail = new GridHadoopTaskStatus(state, e);
+
+        for (GridHadoopTaskInfo task : tasks)
+            jobTracker.onTaskFinished(task, fail);
+    }
+
+    /**
+     * Starts process template that will be ready to execute Hadoop tasks.
+     *
+     * @param job Job instance.
+     * @param plan Map reduce plan.
+     */
+    private HadoopProcess startProcess(final GridHadoopJob job, final GridHadoopMapReducePlan plan) {
+        final UUID childProcId = UUID.randomUUID();
+
+        GridHadoopJobId jobId = job.id();
+
+        final GridHadoopProcessFuture fut = new GridHadoopProcessFuture(childProcId, jobId, ctx.kernalContext());
+
+        final HadoopProcess proc = new HadoopProcess(jobId, fut, plan.reducers(ctx.localNodeId()));
+
+        HadoopProcess old = runningProcsByJobId.put(jobId, proc);
+
+        assert old == null;
+
+        old = runningProcsByProcId.put(childProcId, proc);
+
+        assert old == null;
+
+        ctx.kernalContext().closure().runLocalSafe(new Runnable() {
+            @Override public void run() {
+                if (!busyLock.tryReadLock()) {
+                    fut.onDone(new IgniteCheckedException("Failed to start external process (grid is stopping)."));
+
+                    return;
+                }
+
+                try {
+                    GridHadoopExternalTaskMetadata startMeta = buildTaskMeta();
+
+                    if (log.isDebugEnabled())
+                        log.debug("Created hadoop child process metadata for job [job=" + job +
+                            ", childProcId=" + childProcId + ", taskMeta=" + startMeta + ']');
+
+                    Process proc = startJavaProcess(childProcId, startMeta, job);
+
+                    BufferedReader rdr = new BufferedReader(new InputStreamReader(proc.getInputStream()));
+
+                    String line;
+
+                    // Read up all the process output.
+                    while ((line = rdr.readLine()) != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Tracing process output: " + line);
+
+                        if ("Started".equals(line)) {
+                            // Process started successfully, it should not write anything more to the output stream.
+                            if (log.isDebugEnabled())
+                                log.debug("Successfully started child process [childProcId=" + childProcId +
+                                    ", meta=" + job + ']');
+
+                            fut.onProcessStarted(proc);
+
+                            break;
+                        }
+                        else if ("Failed".equals(line)) {
+                            StringBuilder sb = new StringBuilder("Failed to start child process: " + job + "\n");
+
+                            while ((line = rdr.readLine()) != null)
+                                sb.append("    ").append(line).append("\n");
+
+                            // Cut last character.
+                            sb.setLength(sb.length() - 1);
+
+                            log.warning(sb.toString());
+
+                            fut.onDone(new IgniteCheckedException(sb.toString()));
+
+                            break;
+                        }
+                    }
+                }
+                catch (Throwable e) {
+                    fut.onDone(new IgniteCheckedException("Failed to initialize child process: " + job, e));
+                }
+                finally {
+                    busyLock.readUnlock();
+                }
+            }
+        }, true);
+
+        fut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+            @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                try {
+                    // Make sure there were no exceptions.
+                    f.get();
+
+                    prepareForJob(proc, job, plan);
+                }
+                catch (IgniteCheckedException ignore) {
+                    // Exception is printed in future's onDone() method.
+                }
+            }
+        });
+
+        return proc;
+    }
+
+    /**
+     * Checks that java local command is available.
+     *
+     * @throws IgniteCheckedException If initialization failed.
+     */
+    private void initJavaCommand() throws IgniteCheckedException {
+        String javaHome = System.getProperty("java.home");
+
+        if (javaHome == null)
+            javaHome = System.getenv("JAVA_HOME");
+
+        if (javaHome == null)
+            throw new IgniteCheckedException("Failed to locate JAVA_HOME.");
+
+        javaCmd = javaHome + File.separator + "bin" + File.separator + (U.isWindows() ? "java.exe" : "java");
+
+        try {
+            Process proc = new ProcessBuilder(javaCmd, "-version").redirectErrorStream(true).start();
+
+            Collection<String> out = readProcessOutput(proc);
+
+            int res = proc.waitFor();
+
+            if (res != 0)
+                throw new IgniteCheckedException("Failed to execute 'java -version' command (process finished with nonzero " +
+                    "code) [exitCode=" + res + ", javaCmd='" + javaCmd + "', msg=" + F.first(out) + ']');
+
+            if (log.isInfoEnabled()) {
+                log.info("Will use java for external task execution: ");
+
+                for (String s : out)
+                    log.info("    " + s);
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to check java for external task execution.", e);
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            throw new IgniteCheckedException("Failed to wait for process completion (thread got interrupted).", e);
+        }
+    }
+
+    /**
+     * Reads process output line-by-line.
+     *
+     * @param proc Process to read output.
+     * @return Read lines.
+     * @throws IOException If read failed.
+     */
+    private Collection<String> readProcessOutput(Process proc) throws IOException {
+        BufferedReader rdr = new BufferedReader(new InputStreamReader(proc.getInputStream()));
+
+        Collection<String> res = new ArrayList<>();
+
+        String s;
+
+        while ((s = rdr.readLine()) != null)
+            res.add(s);
+
+        return res;
+    }
+
+    /**
+     * Builds process from metadata.
+     *
+     * @param childProcId Child process ID.
+     * @param startMeta Metadata.
+     * @param job Job.
+     * @return Started process.
+     */
+    private Process startJavaProcess(UUID childProcId, GridHadoopExternalTaskMetadata startMeta,
+        GridHadoopJob job) throws Exception {
+        String outFldr = jobWorkFolder(job.id()) + File.separator + childProcId;
+
+        if (log.isDebugEnabled())
+            log.debug("Will write process log output to: " + outFldr);
+
+        List<String> cmd = new ArrayList<>();
+
+        File workDir = U.resolveWorkDirectory("", false);
+
+        cmd.add(javaCmd);
+        cmd.addAll(startMeta.jvmOptions());
+        cmd.add("-cp");
+        cmd.add(buildClasspath(startMeta.classpath()));
+        cmd.add(GridHadoopExternalProcessStarter.class.getName());
+        cmd.add("-cpid");
+        cmd.add(String.valueOf(childProcId));
+        cmd.add("-ppid");
+        cmd.add(String.valueOf(nodeDesc.processId()));
+        cmd.add("-nid");
+        cmd.add(String.valueOf(nodeDesc.parentNodeId()));
+        cmd.add("-addr");
+        cmd.add(nodeDesc.address());
+        cmd.add("-tport");
+        cmd.add(String.valueOf(nodeDesc.tcpPort()));
+        cmd.add("-sport");
+        cmd.add(String.valueOf(nodeDesc.sharedMemoryPort()));
+        cmd.add("-out");
+        cmd.add(outFldr);
+        cmd.add("-wd");
+        cmd.add(workDir.getAbsolutePath());
+
+        return new ProcessBuilder(cmd)
+            .redirectErrorStream(true)
+            .directory(workDir)
+            .start();
+    }
+
+    /**
+     * Gets job work folder.
+     *
+     * @param jobId Job ID.
+     * @return Job work folder.
+     */
+    private String jobWorkFolder(GridHadoopJobId jobId) {
+        return outputBase + File.separator + "Job_" + jobId;
+    }
+
+    /**
+     * @param cp Classpath collection.
+     * @return Classpath string.
+     */
+    private String buildClasspath(Collection<String> cp) {
+        assert !cp.isEmpty();
+
+        StringBuilder sb = new StringBuilder();
+
+        for (String s : cp)
+            sb.append(s).append(pathSep);
+
+        sb.setLength(sb.length() - 1);
+
+        return sb.toString();
+    }
+
+    /**
+     * Sends job info update request to remote process.
+     *
+     * @param proc Process to send request to.
+     * @param meta Job metadata.
+     */
+    private void sendJobInfoUpdate(HadoopProcess proc, GridHadoopJobMetadata meta) {
+        Map<Integer, GridHadoopProcessDescriptor> rdcAddrs = meta.reducersAddresses();
+
+        int rdcNum = meta.mapReducePlan().reducers();
+
+        GridHadoopProcessDescriptor[] addrs = null;
+
+        if (rdcAddrs != null && rdcAddrs.size() == rdcNum) {
+            addrs = new GridHadoopProcessDescriptor[rdcNum];
+
+            for (int i = 0; i < rdcNum; i++) {
+                GridHadoopProcessDescriptor desc = rdcAddrs.get(i);
+
+                assert desc != null : "Missing reducing address [meta=" + meta + ", rdc=" + i + ']';
+
+                addrs[i] = desc;
+            }
+        }
+
+        try {
+            comm.sendMessage(proc.descriptor(), new GridHadoopJobInfoUpdateRequest(proc.jobId, meta.phase(), addrs));
+        }
+        catch (IgniteCheckedException e) {
+            if (!proc.terminated()) {
+                log.error("Failed to send job state update message to remote child process (will kill the process) " +
+                    "[jobId=" + proc.jobId + ", meta=" + meta + ']', e);
+
+                proc.terminate();
+            }
+        }
+    }
+
+    /**
+     * Sends prepare request to remote process.
+     *
+     * @param proc Process to send request to.
+     * @param job Job.
+     * @param plan Map reduce plan.
+     */
+    private void prepareForJob(HadoopProcess proc, GridHadoopJob job, GridHadoopMapReducePlan plan) {
+        try {
+            comm.sendMessage(proc.descriptor(), new GridHadoopPrepareForJobRequest(job.id(), job.info(),
+                plan.reducers(), plan.reducers(ctx.localNodeId())));
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send job prepare request to remote process [proc=" + proc + ", job=" + job +
+                ", plan=" + plan + ']', e);
+
+            proc.terminate();
+        }
+    }
+
+    /**
+     * Processes task finished message.
+     *
+     * @param desc Remote process descriptor.
+     * @param taskMsg Task finished message.
+     */
+    private void processTaskFinishedMessage(GridHadoopProcessDescriptor desc, GridHadoopTaskFinishedMessage taskMsg) {
+        HadoopProcess proc = runningProcsByProcId.get(desc.processId());
+
+        if (proc != null)
+            proc.removeTask(taskMsg.taskInfo());
+
+        jobTracker.onTaskFinished(taskMsg.taskInfo(), taskMsg.status());
+    }
+
+    /**
+     *
+     */
+    private class MessageListener implements GridHadoopMessageListener {
+        /** {@inheritDoc} */
+        @Override public void onMessageReceived(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
+            if (!busyLock.tryReadLock())
+                return;
+
+            try {
+                if (msg instanceof GridHadoopProcessStartedAck) {
+                    HadoopProcess proc = runningProcsByProcId.get(desc.processId());
+
+                    assert proc != null : "Missing child process for processId: " + desc;
+
+                    GridHadoopProcessFuture fut = proc.initFut;
+
+                    if (fut != null)
+                        fut.onReplyReceived(desc);
+                    // Safety.
+                    else
+                        log.warning("Failed to find process start future (will ignore): " + desc);
+                }
+                else if (msg instanceof GridHadoopTaskFinishedMessage) {
+                    GridHadoopTaskFinishedMessage taskMsg = (GridHadoopTaskFinishedMessage)msg;
+
+                    processTaskFinishedMessage(desc, taskMsg);
+                }
+                else
+                    log.warning("Unexpected message received by node [desc=" + desc + ", msg=" + msg + ']');
+            }
+            finally {
+                busyLock.readUnlock();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onConnectionLost(GridHadoopProcessDescriptor desc) {
+            if (!busyLock.tryReadLock())
+                return;
+
+            try {
+                if (desc == null) {
+                    U.warn(log, "Handshake failed.");
+
+                    return;
+                }
+
+                // Notify job tracker about failed tasks.
+                HadoopProcess proc = runningProcsByProcId.get(desc.processId());
+
+                if (proc != null) {
+                    Collection<GridHadoopTaskInfo> tasks = proc.tasks();
+
+                    if (!F.isEmpty(tasks)) {
+                        log.warning("Lost connection with alive process (will terminate): " + desc);
+
+                        GridHadoopTaskStatus status = new GridHadoopTaskStatus(CRASHED,
+                            new IgniteCheckedException("Failed to run tasks (external process finished unexpectedly): " + desc));
+
+                        for (GridHadoopTaskInfo info : tasks)
+                            jobTracker.onTaskFinished(info, status);
+
+                        runningProcsByJobId.remove(proc.jobId(), proc);
+                    }
+
+                    // Safety.
+                    proc.terminate();
+                }
+            }
+            finally {
+                busyLock.readUnlock();
+            }
+        }
+    }
+
+    /**
+     * Hadoop process.
+     */
+    private static class HadoopProcess extends ReentrantLock {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Job ID. */
+        private final GridHadoopJobId jobId;
+
+        /** Process. */
+        private Process proc;
+
+        /** Init future. Completes when process is ready to receive messages. */
+        private final GridHadoopProcessFuture initFut;
+
+        /** Process descriptor. */
+        private GridHadoopProcessDescriptor procDesc;
+
+        /** Reducers planned for this process. */
+        private Collection<Integer> reducers;
+
+        /** Tasks. */
+        private final Collection<GridHadoopTaskInfo> tasks = new ConcurrentLinkedDeque8<>();
+
+        /** Terminated flag. */
+        private volatile boolean terminated;
+
+        /**
+         * @param jobId Job ID.
+         * @param initFut Init future.
+         */
+        private HadoopProcess(GridHadoopJobId jobId, GridHadoopProcessFuture initFut,
+            int[] reducers) {
+            this.jobId = jobId;
+            this.initFut = initFut;
+
+            if (!F.isEmpty(reducers)) {
+                this.reducers = new ArrayList<>(reducers.length);
+
+                for (int r : reducers)
+                    this.reducers.add(r);
+            }
+        }
+
+        /**
+         * @return Communication process descriptor.
+         */
+        private GridHadoopProcessDescriptor descriptor() {
+            return procDesc;
+        }
+
+        /**
+         * @return Job ID.
+         */
+        public GridHadoopJobId jobId() {
+            return jobId;
+        }
+
+        /**
+         * Initialized callback.
+         *
+         * @param proc Java process representation.
+         * @param procDesc Process descriptor.
+         */
+        private void onInitialized(Process proc, GridHadoopProcessDescriptor procDesc) {
+            this.proc = proc;
+            this.procDesc = procDesc;
+        }
+
+        /**
+         * Terminates process (kills it).
+         */
+        private void terminate() {
+            // Guard against concurrent message sending.
+            lock();
+
+            try {
+                terminated = true;
+
+                if (!initFut.isDone())
+                    initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                        @Override public void apply(
+                            IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                            proc.destroy();
+                        }
+                    });
+                else
+                    proc.destroy();
+            }
+            finally {
+                unlock();
+            }
+        }
+
+        /**
+         * @return Terminated flag.
+         */
+        private boolean terminated() {
+            return terminated;
+        }
+
+        /**
+         * Sets process tasks.
+         *
+         * @param tasks Tasks to set.
+         */
+        private void addTasks(Collection<GridHadoopTaskInfo> tasks) {
+            this.tasks.addAll(tasks);
+        }
+
+        /**
+         * Removes task when it was completed.
+         *
+         * @param task Task to remove.
+         */
+        private void removeTask(GridHadoopTaskInfo task) {
+            if (tasks != null)
+                tasks.remove(task);
+        }
+
+        /**
+         * @return Collection of tasks.
+         */
+        private Collection<GridHadoopTaskInfo> tasks() {
+            return tasks;
+        }
+
+        /**
+         * @return Planned reducers.
+         */
+        private Collection<Integer> reducers() {
+            return reducers;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(HadoopProcess.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private class GridHadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Child process ID. */
+        private UUID childProcId;
+
+        /** Job ID. */
+        private GridHadoopJobId jobId;
+
+        /** Process descriptor. */
+        private GridHadoopProcessDescriptor desc;
+
+        /** Running process. */
+        private Process proc;
+
+        /** Process started flag. */
+        private volatile boolean procStarted;
+
+        /** Reply received flag. */
+        private volatile boolean replyReceived;
+
+        /** Logger. */
+        private final IgniteLogger log = HadoopExternalTaskExecutor.this.log;
+
+        /**
+         * Empty constructor.
+         */
+        public GridHadoopProcessFuture() {
+            // No-op.
+        }
+
+        /**
+         * @param ctx Kernal context.
+         */
+        private GridHadoopProcessFuture(UUID childProcId, GridHadoopJobId jobId, GridKernalContext ctx) {
+            super(ctx);
+
+            this.childProcId = childProcId;
+            this.jobId = jobId;
+        }
+
+        /**
+         * Process started callback.
+         */
+        public void onProcessStarted(Process proc) {
+            this.proc = proc;
+
+            procStarted = true;
+
+            if (procStarted && replyReceived)
+                onDone(F.t(proc, desc));
+        }
+
+        /**
+         * Reply received callback.
+         */
+        public void onReplyReceived(GridHadoopProcessDescriptor desc) {
+            assert childProcId.equals(desc.processId());
+
+            this.desc = desc;
+
+            replyReceived = true;
+
+            if (procStarted && replyReceived)
+                onDone(F.t(proc, desc));
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable IgniteBiTuple<Process, GridHadoopProcessDescriptor> res,
+            @Nullable Throwable err) {
+            if (err == null) {
+                HadoopProcess proc = runningProcsByProcId.get(childProcId);
+
+                assert proc != null;
+
+                assert proc.initFut == this;
+
+                proc.onInitialized(res.get1(), res.get2());
+
+                if (!F.isEmpty(proc.reducers()))
+                    jobTracker.onExternalMappersInitialized(jobId, proc.reducers(), desc);
+            }
+            else {
+                // Clean up since init failed.
+                runningProcsByJobId.remove(jobId);
+                runningProcsByProcId.remove(childProcId);
+            }
+
+            if (super.onDone(res, err)) {
+                if (err == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Initialized child process for external task execution [jobId=" + jobId +
+                            ", desc=" + desc + ", initTime=" + duration() + ']');
+                }
+                else
+                    U.error(log, "Failed to initialize child process for external task execution [jobId=" + jobId +
+                        ", desc=" + desc + ']', err);
+
+                return true;
+            }
+
+            return false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
index 878b61b..da59483 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1MapTask.java
@@ -86,7 +86,7 @@ public class GridHadoopV1MapTask extends GridHadoopV1Task {
                 try {
                     while (reader.next(key, val)) {
                         if (isCancelled())
-                            throw new GridHadoopTaskCancelledException("Map task cancelled.");
+                            throw new HadoopTaskCancelledException("Map task cancelled.");
 
                         mapper.map(key, val, collector, reporter);
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
index 7deea90..3aca637 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1ReduceTask.java
@@ -67,7 +67,7 @@ public class GridHadoopV1ReduceTask extends GridHadoopV1Task {
                 try {
                     while (input.next()) {
                         if (isCancelled())
-                            throw new GridHadoopTaskCancelledException("Reduce task cancelled.");
+                            throw new HadoopTaskCancelledException("Reduce task cancelled.");
 
                         reducer.reduce(input.key(), input.values(), collector, Reporter.NULL);
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
index 257f4ea..0e1fb44 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Splitter.java
@@ -58,7 +58,7 @@ public class GridHadoopV1Splitter {
                     res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
                 }
                 else
-                    res.add(GridHadoopUtils.wrapSplit(i, nativeSplit, nativeSplit.getLocations()));
+                    res.add(HadoopUtils.wrapSplit(i, nativeSplit, nativeSplit.getLocations()));
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
index 86a7264..305bc4e 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Task.java
@@ -72,7 +72,7 @@ public abstract class GridHadoopV1Task extends GridHadoopTask {
             /** {@inheritDoc} */
             @Override public void collect(Object key, Object val) throws IOException {
                 if (cancelled)
-                    throw new GridHadoopTaskCancelledException("Task cancelled.");
+                    throw new HadoopTaskCancelledException("Task cancelled.");
 
                 super.collect(key, val);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
index 287b10f..160e34b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
@@ -92,7 +92,7 @@ public class GridHadoopV2Context extends JobContextImpl implements MapContext, R
             else if (split instanceof GridHadoopExternalSplit)
                 throw new UnsupportedOperationException(); // TODO
             else if (split instanceof GridHadoopSplitWrapper)
-                inputSplit = (InputSplit)GridHadoopUtils.unwrapSplit((GridHadoopSplitWrapper)split);
+                inputSplit = (InputSplit) HadoopUtils.unwrapSplit((GridHadoopSplitWrapper) split);
             else
                 throw new IllegalStateException();
         }
@@ -103,7 +103,7 @@ public class GridHadoopV2Context extends JobContextImpl implements MapContext, R
     /** {@inheritDoc} */
     @Override public boolean nextKeyValue() throws IOException, InterruptedException {
         if (cancelled)
-            throw new GridHadoopTaskCancelledException("Task cancelled.");
+            throw new HadoopTaskCancelledException("Task cancelled.");
 
         return reader.nextKeyValue();
     }
@@ -125,7 +125,7 @@ public class GridHadoopV2Context extends JobContextImpl implements MapContext, R
     @SuppressWarnings("unchecked")
     @Override public void write(Object key, Object val) throws IOException, InterruptedException {
         if (cancelled)
-            throw new GridHadoopTaskCancelledException("Task cancelled.");
+            throw new HadoopTaskCancelledException("Task cancelled.");
 
         if (writer != null)
             writer.write(key, val);
@@ -191,7 +191,7 @@ public class GridHadoopV2Context extends JobContextImpl implements MapContext, R
     /** {@inheritDoc} */
     @Override public boolean nextKey() throws IOException, InterruptedException {
         if (cancelled)
-            throw new GridHadoopTaskCancelledException("Task cancelled.");
+            throw new HadoopTaskCancelledException("Task cancelled.");
 
         return input.next();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
index 7c36948..5f1af22 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
@@ -39,7 +39,7 @@ import java.util.*;
 import java.util.Queue;
 import java.util.concurrent.*;
 
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Hadoop job implementation for v2 API.
@@ -81,7 +81,7 @@ public class GridHadoopV2Job implements GridHadoopJob {
      * @param jobInfo Job info.
      * @param log Logger.
      */
-    public GridHadoopV2Job(GridHadoopJobId jobId, final GridHadoopDefaultJobInfo jobInfo, IgniteLogger log) {
+    public GridHadoopV2Job(GridHadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log) {
         assert jobId != null;
         assert jobInfo != null;
 
@@ -90,7 +90,7 @@ public class GridHadoopV2Job implements GridHadoopJob {
 
         hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
 
-        GridHadoopClassLoader clsLdr = (GridHadoopClassLoader)getClass().getClassLoader();
+        HadoopClassLoader clsLdr = (HadoopClassLoader)getClass().getClassLoader();
 
         // Before create JobConf instance we should set new context class loader.
         Thread.currentThread().setContextClassLoader(clsLdr);
@@ -196,7 +196,7 @@ public class GridHadoopV2Job implements GridHadoopJob {
         try {
             if (cls == null) {
                 // If there is no pooled class, then load new one.
-                GridHadoopClassLoader ldr = new GridHadoopClassLoader(rsrcMgr.classPath());
+                HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath());
 
                 cls = ldr.loadClass(GridHadoopV2TaskContext.class.getName());
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
index e8ce70b..68338a6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
@@ -59,7 +59,7 @@ public class GridHadoopV2Splitter {
                     res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
                 }
                 else
-                    res.add(GridHadoopUtils.wrapSplit(id, nativeSplit, nativeSplit.getLocations()));
+                    res.add(HadoopUtils.wrapSplit(id, nativeSplit, nativeSplit.getLocations()));
 
                 id++;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
index 82be91f..3e88362 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
@@ -40,7 +40,7 @@ import java.io.*;
 import java.util.*;
 
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Context for task execution.
@@ -186,7 +186,7 @@ public class GridHadoopV2TaskContext extends GridHadoopTaskContext {
             }
 
             if (cancelled)
-                throw new GridHadoopTaskCancelledException("Task cancelled.");
+                throw new HadoopTaskCancelledException("Task cancelled.");
 
             try {
                 task.run(this);


[06/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Posted by vo...@apache.org.
# IGNITE-386: WIP on internal namings.


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

Branch: refs/heads/ignite-386
Commit: ace354c6cfe9293a786ad3c7aaea94ab6abcfd0b
Parents: 7d46deb
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 3 15:43:58 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 3 15:43:58 2015 +0300

----------------------------------------------------------------------
 bin/setup-hadoop.bat                            |    2 +-
 bin/setup-hadoop.sh                             |    2 +-
 .../hadoop/GridHadoopClassLoader.java           |  552 ------
 .../processors/hadoop/GridHadoopComponent.java  |   61 -
 .../processors/hadoop/GridHadoopContext.java    |  196 ---
 .../hadoop/GridHadoopDefaultJobInfo.java        |  163 --
 .../processors/hadoop/GridHadoopImpl.java       |  132 --
 .../processors/hadoop/GridHadoopSetup.java      |  505 ------
 .../GridHadoopTaskCancelledException.java       |   35 -
 .../processors/hadoop/GridHadoopUtils.java      |  308 ----
 .../processors/hadoop/HadoopClassLoader.java    |  552 ++++++
 .../processors/hadoop/HadoopComponent.java      |   61 +
 .../processors/hadoop/HadoopContext.java        |  196 +++
 .../processors/hadoop/HadoopCounterGroup.java   |    2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |  163 ++
 .../internal/processors/hadoop/HadoopImpl.java  |  132 ++
 .../internal/processors/hadoop/HadoopSetup.java |  505 ++++++
 .../hadoop/HadoopTaskCancelledException.java    |   35 +
 .../internal/processors/hadoop/HadoopUtils.java |  308 ++++
 .../hadoop/IgniteHadoopProcessor.java           |   32 +-
 .../counter/GridHadoopFSCounterWriter.java      |    2 +-
 .../counter/GridHadoopPerformanceCounter.java   |    2 +-
 .../hadoop/jobtracker/GridHadoopJobTracker.java | 1625 ------------------
 .../hadoop/jobtracker/HadoopJobTracker.java     | 1625 ++++++++++++++++++
 .../proto/GridHadoopProtocolSubmitJobTask.java  |    2 +-
 .../hadoop/proto/HadoopClientProtocol.java      |    6 +-
 .../hadoop/shuffle/GridHadoopShuffle.java       |  256 ---
 .../hadoop/shuffle/HadoopShuffle.java           |  256 +++
 .../GridHadoopEmbeddedTaskExecutor.java         |  146 --
 .../taskexecutor/GridHadoopRunnableTask.java    |    4 +-
 .../GridHadoopTaskExecutorAdapter.java          |   57 -
 .../HadoopEmbeddedTaskExecutor.java             |  146 ++
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   57 +
 .../GridHadoopExternalTaskExecutor.java         |  960 -----------
 .../external/HadoopExternalTaskExecutor.java    |  960 +++++++++++
 .../hadoop/v1/GridHadoopV1MapTask.java          |    2 +-
 .../hadoop/v1/GridHadoopV1ReduceTask.java       |    2 +-
 .../hadoop/v1/GridHadoopV1Splitter.java         |    2 +-
 .../processors/hadoop/v1/GridHadoopV1Task.java  |    2 +-
 .../hadoop/v2/GridHadoopV2Context.java          |    8 +-
 .../processors/hadoop/v2/GridHadoopV2Job.java   |    8 +-
 .../hadoop/v2/GridHadoopV2Splitter.java         |    2 +-
 .../hadoop/v2/GridHadoopV2TaskContext.java      |    4 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |    4 +-
 .../hadoop/GridHadoopClassLoaderTest.java       |   69 -
 .../hadoop/GridHadoopCommandLineTest.java       |    4 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |    2 +-
 .../hadoop/GridHadoopGroupingTest.java          |    2 +-
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |    2 +-
 .../GridHadoopMapReduceEmbeddedSelfTest.java    |    2 +-
 .../hadoop/GridHadoopMapReduceTest.java         |    2 +-
 .../hadoop/GridHadoopSortingTest.java           |    2 +-
 .../hadoop/GridHadoopSplitWrapperSelfTest.java  |    4 +-
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |    4 +-
 .../hadoop/GridHadoopTasksV1Test.java           |    4 +-
 .../hadoop/GridHadoopTasksV2Test.java           |    4 +-
 .../hadoop/GridHadoopTestTaskContext.java       |    2 +-
 .../hadoop/GridHadoopV2JobSelfTest.java         |    2 +-
 .../hadoop/HadoopClassLoaderTest.java           |   69 +
 ...GridHadoopExternalTaskExecutionSelfTest.java |    2 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |    2 +-
 62 files changed, 5130 insertions(+), 5130 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/bin/setup-hadoop.bat
----------------------------------------------------------------------
diff --git a/bin/setup-hadoop.bat b/bin/setup-hadoop.bat
index c4c73b3..a11ef8c 100644
--- a/bin/setup-hadoop.bat
+++ b/bin/setup-hadoop.bat
@@ -23,6 +23,6 @@
 
 if "%OS%" == "Windows_NT" setlocal
 
-set MAIN_CLASS=org.apache.ignite.hadoop.GridHadoopSetup
+set MAIN_CLASS=org.apache.ignite.internal.processors.hadoop.HadoopSetup
 
 call "%~dp0\ignite.bat" %*

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/bin/setup-hadoop.sh
----------------------------------------------------------------------
diff --git a/bin/setup-hadoop.sh b/bin/setup-hadoop.sh
index 8969dfa..d66353f 100755
--- a/bin/setup-hadoop.sh
+++ b/bin/setup-hadoop.sh
@@ -54,7 +54,7 @@ setIgniteHome
 #
 # Set utility environment.
 #
-export MAIN_CLASS=org.apache.ignite.internal.processors.hadoop.GridHadoopSetup
+export MAIN_CLASS=org.apache.ignite.internal.processors.hadoop.HadoopSetup
 
 #
 # Start utility.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoader.java
deleted file mode 100644
index bc4c0bb..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopClassLoader.java
+++ /dev/null
@@ -1,552 +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.processors.hadoop.v2.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-import org.objectweb.asm.*;
-import org.objectweb.asm.commons.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Class loader allowing explicitly load classes without delegation to parent class loader.
- * Also supports class parsing for finding dependencies which contain transitive dependencies
- * unavailable for parent.
- */
-public class GridHadoopClassLoader extends URLClassLoader {
-    /**
-     * We are very parallel capable.
-     */
-    static {
-        registerAsParallelCapable();
-    }
-
-    /** */
-    private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)GridHadoopClassLoader.class.getClassLoader();
-
-    /** */
-    private static final Collection<URL> appJars = F.asList(APP_CLS_LDR.getURLs());
-
-    /** */
-    private static volatile Collection<URL> hadoopJars;
-
-    /** */
-    private static final Map<String, Boolean> cache = new ConcurrentHashMap8<>();
-
-    /** */
-    private static final Map<String, byte[]> bytesCache = new ConcurrentHashMap8<>();
-
-    /**
-     * @param urls Urls.
-     */
-    public GridHadoopClassLoader(URL[] urls) {
-        super(addHadoopUrls(urls), APP_CLS_LDR);
-
-        assert !(getParent() instanceof GridHadoopClassLoader);
-    }
-
-    /**
-     * Need to parse only Ignite Hadoop and IGFS classes.
-     *
-     * @param cls Class name.
-     * @return {@code true} if we need to check this class.
-     */
-    private static boolean isIgfsHadoop(String cls) {
-        String ignitePackagePrefix = "org.apache.ignite";
-        int len = ignitePackagePrefix.length();
-
-        return cls.startsWith(ignitePackagePrefix) && (cls.indexOf("igfs.", len) != -1 || cls.indexOf(".fs.", len) != -1 || cls.indexOf("hadoop.", len) != -1);
-    }
-
-    /**
-     * @param cls Class name.
-     * @return {@code true} If this is Hadoop class.
-     */
-    private static boolean isHadoop(String cls) {
-        return cls.startsWith("org.apache.hadoop.");
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
-        try {
-            if (isHadoop(name)) { // Always load Hadoop classes explicitly, since Hadoop can be available in App classpath.
-                if (name.endsWith(".util.ShutdownHookManager"))  // Dirty hack to get rid of Hadoop shutdown hooks.
-                    return loadFromBytes(name, GridHadoopShutdownHookManager.class.getName());
-                else if (name.endsWith(".util.NativeCodeLoader"))
-                    return loadFromBytes(name, GridHadoopNativeCodeLoader.class.getName());
-
-                return loadClassExplicitly(name, resolve);
-            }
-
-            if (isIgfsHadoop(name)) { // For Ignite Hadoop and IGFS classes we have to check if they depend on Hadoop.
-                Boolean hasDeps = cache.get(name);
-
-                if (hasDeps == null) {
-                    hasDeps = hasExternalDependencies(name, new HashSet<String>());
-
-                    cache.put(name, hasDeps);
-                }
-
-                if (hasDeps)
-                    return loadClassExplicitly(name, resolve);
-            }
-
-            return super.loadClass(name, resolve);
-        }
-        catch (NoClassDefFoundError | ClassNotFoundException e) {
-            throw new ClassNotFoundException("Failed to load class: " + name, e);
-        }
-    }
-
-    /**
-     * @param name Name.
-     * @param replace Replacement.
-     * @return Class.
-     */
-    private Class<?> loadFromBytes(final String name, final String replace) {
-        synchronized (getClassLoadingLock(name)) {
-            // First, check if the class has already been loaded
-            Class c = findLoadedClass(name);
-
-            if (c != null)
-                return c;
-
-            byte[] bytes = bytesCache.get(name);
-
-            if (bytes == null) {
-                InputStream in = loadClassBytes(getParent(), replace);
-
-                ClassReader rdr;
-
-                try {
-                    rdr = new ClassReader(in);
-                }
-                catch (IOException e) {
-                    throw new RuntimeException(e);
-                }
-
-                ClassWriter w = new ClassWriter(Opcodes.ASM4);
-
-                rdr.accept(new RemappingClassAdapter(w, new Remapper() {
-                    /** */
-                    String replaceType = replace.replace('.', '/');
-
-                    /** */
-                    String nameType = name.replace('.', '/');
-
-                    @Override public String map(String type) {
-                        if (type.equals(replaceType))
-                            return nameType;
-
-                        return type;
-                    }
-                }), ClassReader.EXPAND_FRAMES);
-
-                bytes = w.toByteArray();
-
-                bytesCache.put(name, bytes);
-            }
-
-            return defineClass(name, bytes, 0, bytes.length);
-        }
-    }
-
-    /**
-     * @param name Class name.
-     * @param resolve Resolve class.
-     * @return Class.
-     * @throws ClassNotFoundException If failed.
-     */
-    private Class<?> loadClassExplicitly(String name, boolean resolve) throws ClassNotFoundException {
-        synchronized (getClassLoadingLock(name)) {
-            // First, check if the class has already been loaded
-            Class c = findLoadedClass(name);
-
-            if (c == null) {
-                long t1 = System.nanoTime();
-
-                c = findClass(name);
-
-                // this is the defining class loader; record the stats
-                sun.misc.PerfCounter.getFindClassTime().addElapsedTimeFrom(t1);
-                sun.misc.PerfCounter.getFindClasses().increment();
-            }
-
-            if (resolve)
-                resolveClass(c);
-
-            return c;
-        }
-    }
-
-    /**
-     * @param ldr Loader.
-     * @param clsName Class.
-     * @return Input stream.
-     */
-    @Nullable private InputStream loadClassBytes(ClassLoader ldr, String clsName) {
-        return ldr.getResourceAsStream(clsName.replace('.', '/') + ".class");
-    }
-
-    /**
-     * @param clsName Class name.
-     * @return {@code true} If the class has external dependencies.
-     */
-    boolean hasExternalDependencies(final String clsName, final Set<String> visited) {
-        if (isHadoop(clsName)) // Hadoop must not be in classpath but Idea sucks, so filtering explicitly as external.
-            return true;
-
-        // Try to get from parent to check if the type accessible.
-        InputStream in = loadClassBytes(getParent(), clsName);
-
-        if (in == null) // The class is external itself, it must be loaded from this class loader.
-            return true;
-
-        if (!isIgfsHadoop(clsName)) // Other classes should not have external dependencies.
-            return false;
-
-        final ClassReader rdr;
-
-        try {
-            rdr = new ClassReader(in);
-        }
-        catch (IOException e) {
-            throw new RuntimeException("Failed to read class: " + clsName, e);
-        }
-
-        visited.add(clsName);
-
-        final AtomicBoolean hasDeps = new AtomicBoolean();
-
-        rdr.accept(new ClassVisitor(Opcodes.ASM4) {
-            AnnotationVisitor av = new AnnotationVisitor(Opcodes.ASM4) {
-                // TODO
-            };
-
-            FieldVisitor fv = new FieldVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-            };
-
-            MethodVisitor mv = new MethodVisitor(Opcodes.ASM4) {
-                @Override public AnnotationVisitor visitAnnotation(String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-
-                @Override public AnnotationVisitor visitParameterAnnotation(int i, String desc, boolean b) {
-                    onType(desc);
-
-                    return av;
-                }
-
-                @Override public AnnotationVisitor visitAnnotationDefault() {
-                    return av;
-                }
-
-                @Override public void visitFieldInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                    onType(desc);
-                }
-
-                @Override public void visitFrame(int i, int i2, Object[] locTypes, int i3, Object[] stackTypes) {
-                    for (Object o : locTypes) {
-                        if (o instanceof String)
-                            onType((String)o);
-                    }
-
-                    for (Object o : stackTypes) {
-                        if (o instanceof String)
-                            onType((String)o);
-                    }
-                }
-
-                @Override public void visitLocalVariable(String name, String desc, String signature, Label lb,
-                    Label lb2, int i) {
-                    onType(desc);
-                }
-
-                @Override public void visitMethodInsn(int i, String owner, String name, String desc) {
-                    onType(owner);
-                }
-
-                @Override public void visitMultiANewArrayInsn(String desc, int dim) {
-                    onType(desc);
-                }
-
-                @Override public void visitTryCatchBlock(Label lb, Label lb2, Label lb3, String e) {
-                    onType(e);
-                }
-            };
-
-            void onClass(String depCls) {
-                assert validateClassName(depCls) : depCls;
-
-                if (depCls.startsWith("java.")) // Filter out platform classes.
-                    return;
-
-                if (visited.contains(depCls))
-                    return;
-
-                Boolean res = cache.get(depCls);
-
-                if (res == Boolean.TRUE || (res == null && hasExternalDependencies(depCls, visited)))
-                    hasDeps.set(true);
-            }
-
-            void onType(String type) {
-                if (type == null)
-                    return;
-
-                int off = 0;
-
-                while (type.charAt(off) == '[')
-                    off++; // Handle arrays.
-
-                if (off != 0)
-                    type = type.substring(off);
-
-                if (type.length() == 1)
-                    return; // Get rid of primitives.
-
-                if (type.charAt(type.length() - 1) == ';') {
-                    assert type.charAt(0) == 'L' : type;
-
-                    type = type.substring(1, type.length() - 1);
-                }
-
-                type = type.replace('/', '.');
-
-                onClass(type);
-            }
-
-            @Override public void visit(int i, int i2, String name, String signature, String superName,
-                String[] ifaces) {
-                onType(superName);
-
-                if (ifaces != null) {
-                    for (String iface : ifaces)
-                        onType(iface);
-                }
-            }
-
-            @Override public AnnotationVisitor visitAnnotation(String desc, boolean visible) {
-                onType(desc);
-
-                return av;
-            }
-
-            @Override public void visitInnerClass(String name, String outerName, String innerName, int i) {
-                onType(name);
-            }
-
-            @Override public FieldVisitor visitField(int i, String name, String desc, String signature, Object val) {
-                onType(desc);
-
-                return fv;
-            }
-
-            @Override public MethodVisitor visitMethod(int i, String name, String desc, String signature,
-                String[] exceptions) {
-                if (exceptions != null) {
-                    for (String e : exceptions)
-                        onType(e);
-                }
-
-                return mv;
-            }
-        }, 0);
-
-        if (hasDeps.get()) // We already know that we have dependencies, no need to check parent.
-            return true;
-
-        // Here we are known to not have any dependencies but possibly we have a parent which have them.
-        int idx = clsName.lastIndexOf('$');
-
-        if (idx == -1) // No parent class.
-            return false;
-
-        String parentCls = clsName.substring(0, idx);
-
-        if (visited.contains(parentCls))
-            return false;
-
-        Boolean res = cache.get(parentCls);
-
-        if (res == null)
-            res = hasExternalDependencies(parentCls, visited);
-
-        return res;
-    }
-
-    /**
-     * @param name Class name.
-     * @return {@code true} If this is a valid class name.
-     */
-    private static boolean validateClassName(String name) {
-        int len = name.length();
-
-        if (len <= 1)
-            return false;
-
-        if (!Character.isJavaIdentifierStart(name.charAt(0)))
-            return false;
-
-        boolean hasDot = false;
-
-        for (int i = 1; i < len; i++) {
-            char c = name.charAt(i);
-
-            if (c == '.')
-                hasDot = true;
-            else if (!Character.isJavaIdentifierPart(c))
-                return false;
-        }
-
-        return hasDot;
-    }
-
-    /**
-     * @param name Variable name.
-     * @param dflt Default.
-     * @return Value.
-     */
-    private static String getEnv(String name, String dflt) {
-        String res = System.getProperty(name);
-
-        if (F.isEmpty(res))
-            res = System.getenv(name);
-
-        return F.isEmpty(res) ? dflt : res;
-    }
-
-    /**
-     * @param res Result.
-     * @param dir Directory.
-     * @param startsWith Starts with prefix.
-     * @throws MalformedURLException If failed.
-     */
-    private static void addUrls(Collection<URL> res, File dir, final String startsWith) throws Exception {
-        File[] files = dir.listFiles(new FilenameFilter() {
-            @Override public boolean accept(File dir, String name) {
-                return startsWith == null || name.startsWith(startsWith);
-            }
-        });
-
-        if (files == null)
-            throw new IOException("Path is not a directory: " + dir);
-
-        for (File file : files)
-            res.add(file.toURI().toURL());
-    }
-
-    /**
-     * @param urls URLs.
-     * @return URLs.
-     */
-    private static URL[] addHadoopUrls(URL[] urls) {
-        Collection<URL> hadoopJars;
-
-        try {
-            hadoopJars = hadoopUrls();
-        }
-        catch (IgniteCheckedException e) {
-            throw new RuntimeException(e);
-        }
-
-        ArrayList<URL> list = new ArrayList<>(hadoopJars.size() + appJars.size() + (urls == null ? 0 : urls.length));
-
-        list.addAll(appJars);
-        list.addAll(hadoopJars);
-
-        if (!F.isEmpty(urls))
-            list.addAll(F.asList(urls));
-
-        return list.toArray(new URL[list.size()]);
-    }
-
-    /**
-     * @return HADOOP_HOME Variable.
-     */
-    @Nullable public static String hadoopHome() {
-        return getEnv("HADOOP_PREFIX", getEnv("HADOOP_HOME", null));
-    }
-
-    /**
-     * @return Collection of jar URLs.
-     * @throws IgniteCheckedException If failed.
-     */
-    public static Collection<URL> hadoopUrls() throws IgniteCheckedException {
-        Collection<URL> hadoopUrls = hadoopJars;
-
-        if (hadoopUrls != null)
-            return hadoopUrls;
-
-        synchronized (GridHadoopClassLoader.class) {
-            hadoopUrls = hadoopJars;
-
-            if (hadoopUrls != null)
-                return hadoopUrls;
-
-            hadoopUrls = new ArrayList<>();
-
-            String hadoopPrefix = hadoopHome();
-
-            if (F.isEmpty(hadoopPrefix))
-                throw new IgniteCheckedException("Failed resolve Hadoop installation location. Either HADOOP_PREFIX or " +
-                    "HADOOP_HOME environment variables must be set.");
-
-            String commonHome = getEnv("HADOOP_COMMON_HOME", hadoopPrefix + "/share/hadoop/common");
-            String hdfsHome = getEnv("HADOOP_HDFS_HOME", hadoopPrefix + "/share/hadoop/hdfs");
-            String mapredHome = getEnv("HADOOP_MAPRED_HOME", hadoopPrefix + "/share/hadoop/mapreduce");
-
-            try {
-                addUrls(hadoopUrls, new File(commonHome + "/lib"), null);
-                addUrls(hadoopUrls, new File(hdfsHome + "/lib"), null);
-                addUrls(hadoopUrls, new File(mapredHome + "/lib"), null);
-
-                addUrls(hadoopUrls, new File(hdfsHome), "hadoop-hdfs-");
-
-                addUrls(hadoopUrls, new File(commonHome), "hadoop-common-");
-                addUrls(hadoopUrls, new File(commonHome), "hadoop-auth-");
-                addUrls(hadoopUrls, new File(commonHome + "/lib"), "hadoop-auth-");
-
-                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-common");
-                addUrls(hadoopUrls, new File(mapredHome), "hadoop-mapreduce-client-core");
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException(e);
-            }
-
-            hadoopJars = hadoopUrls;
-
-            return hadoopUrls;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopComponent.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopComponent.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopComponent.java
deleted file mode 100644
index 337bfe9..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopComponent.java
+++ /dev/null
@@ -1,61 +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.*;
-
-/**
- * Abstract class for all hadoop components.
- */
-public abstract class GridHadoopComponent {
-    /** Hadoop context. */
-    protected GridHadoopContext ctx;
-
-    /** Logger. */
-    protected IgniteLogger log;
-
-    /**
-     * @param ctx Hadoop context.
-     */
-    public void start(GridHadoopContext ctx) throws IgniteCheckedException {
-        this.ctx = ctx;
-
-        log = ctx.kernalContext().log(getClass());
-    }
-
-    /**
-     * Stops manager.
-     */
-    public void stop(boolean cancel) {
-        // No-op.
-    }
-
-    /**
-     * Callback invoked when all grid components are started.
-     */
-    public void onKernalStart() throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /**
-     * Callback invoked before all grid components are stopped.
-     */
-    public void onKernalStop(boolean cancel) {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopContext.java
deleted file mode 100644
index 3160e3d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopContext.java
+++ /dev/null
@@ -1,196 +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.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-
-/**
- * Hadoop accelerator context.
- */
-public class GridHadoopContext {
-    /** Kernal context. */
-    private GridKernalContext ctx;
-
-    /** Hadoop configuration. */
-    private GridHadoopConfiguration cfg;
-
-    /** Job tracker. */
-    private GridHadoopJobTracker jobTracker;
-
-    /** External task executor. */
-    private GridHadoopTaskExecutorAdapter taskExecutor;
-
-    /** */
-    private GridHadoopShuffle shuffle;
-
-    /** Managers list. */
-    private List<GridHadoopComponent> components = new ArrayList<>();
-
-    /**
-     * @param ctx Kernal context.
-     */
-    public GridHadoopContext(
-        GridKernalContext ctx,
-        GridHadoopConfiguration cfg,
-        GridHadoopJobTracker jobTracker,
-        GridHadoopTaskExecutorAdapter taskExecutor,
-        GridHadoopShuffle shuffle
-    ) {
-        this.ctx = ctx;
-        this.cfg = cfg;
-
-        this.jobTracker = add(jobTracker);
-        this.taskExecutor = add(taskExecutor);
-        this.shuffle = add(shuffle);
-    }
-
-    /**
-     * Gets list of managers.
-     *
-     * @return List of managers.
-     */
-    public List<GridHadoopComponent> components() {
-        return components;
-    }
-
-    /**
-     * Gets kernal context.
-     *
-     * @return Grid kernal context instance.
-     */
-    public GridKernalContext kernalContext() {
-        return ctx;
-    }
-
-    /**
-     * Gets Hadoop configuration.
-     *
-     * @return Hadoop configuration.
-     */
-    public GridHadoopConfiguration configuration() {
-        return cfg;
-    }
-
-    /**
-     * Gets local node ID. Shortcut for {@code kernalContext().localNodeId()}.
-     *
-     * @return Local node ID.
-     */
-    public UUID localNodeId() {
-        return ctx.localNodeId();
-    }
-
-    /**
-     * Gets local node order.
-     *
-     * @return Local node order.
-     */
-    public long localNodeOrder() {
-        assert ctx.discovery() != null;
-
-        return ctx.discovery().localNode().order();
-    }
-
-    /**
-     * @return Hadoop-enabled nodes.
-     */
-    public Collection<ClusterNode> nodes() {
-        return ctx.discovery().cacheNodes(CU.SYS_CACHE_HADOOP_MR, ctx.discovery().topologyVersion());
-    }
-
-    /**
-     * @return {@code True} if
-     */
-    public boolean jobUpdateLeader() {
-        long minOrder = Long.MAX_VALUE;
-        ClusterNode minOrderNode = null;
-
-        for (ClusterNode node : nodes()) {
-            if (node.order() < minOrder) {
-                minOrder = node.order();
-                minOrderNode = node;
-            }
-        }
-
-        assert minOrderNode != null;
-
-        return localNodeId().equals(minOrderNode.id());
-    }
-
-    /**
-     * @param meta Job metadata.
-     * @return {@code true} If local node is participating in job execution.
-     */
-    public boolean isParticipating(GridHadoopJobMetadata meta) {
-        UUID locNodeId = localNodeId();
-
-        if (locNodeId.equals(meta.submitNodeId()))
-            return true;
-
-        GridHadoopMapReducePlan plan = meta.mapReducePlan();
-
-        return plan.mapperNodeIds().contains(locNodeId) || plan.reducerNodeIds().contains(locNodeId) || jobUpdateLeader();
-    }
-
-    /**
-     * @return Jon tracker instance.
-     */
-    public GridHadoopJobTracker jobTracker() {
-        return jobTracker;
-    }
-
-    /**
-     * @return Task executor.
-     */
-    public GridHadoopTaskExecutorAdapter taskExecutor() {
-        return taskExecutor;
-    }
-
-    /**
-     * @return Shuffle.
-     */
-    public GridHadoopShuffle shuffle() {
-        return shuffle;
-    }
-
-    /**
-     * @return Map-reduce planner.
-     */
-    public GridHadoopMapReducePlanner planner() {
-        return cfg.getMapReducePlanner();
-    }
-
-    /**
-     * Adds component.
-     *
-     * @param c Component to add.
-     * @return Added manager.
-     */
-    private <C extends GridHadoopComponent> C add(C c) {
-        components.add(c);
-
-        return c;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultJobInfo.java
deleted file mode 100644
index 555c573..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultJobInfo.java
+++ /dev/null
@@ -1,163 +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.processors.hadoop.v2.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.util.*;
-
-/**
- * Hadoop job info based on default Hadoop configuration.
- */
-public class GridHadoopDefaultJobInfo implements GridHadoopJobInfo, Externalizable {
-    /** */
-    private static final long serialVersionUID = 5489900236464999951L;
-
-    /** {@code true} If job has combiner. */
-    private boolean hasCombiner;
-
-    /** Number of reducers configured for job. */
-    private int numReduces;
-
-    /** Configuration. */
-    private Map<String,String> props = new HashMap<>();
-
-    /** Job name. */
-    private String jobName;
-
-    /** User name. */
-    private String user;
-
-    /** */
-    private static volatile Class<?> jobCls;
-
-    /**
-     * Default constructor required by {@link Externalizable}.
-     */
-    public GridHadoopDefaultJobInfo() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param jobName Job name.
-     * @param user User name.
-     * @param hasCombiner {@code true} If job has combiner.
-     * @param numReduces Number of reducers configured for job.
-     * @param props All other properties of the job.
-     */
-    public GridHadoopDefaultJobInfo(String jobName, String user, boolean hasCombiner, int numReduces,
-        Map<String, String> props) {
-        this.jobName = jobName;
-        this.user = user;
-        this.hasCombiner = hasCombiner;
-        this.numReduces = numReduces;
-        this.props = props;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String property(String name) {
-        return props.get(name);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJob createJob(GridHadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
-        try {
-            Class<?> jobCls0 = jobCls;
-
-            if (jobCls0 == null) { // It is enough to have only one class loader with only Hadoop classes.
-                synchronized (GridHadoopDefaultJobInfo.class) {
-                    if ((jobCls0 = jobCls) == null) {
-                        GridHadoopClassLoader ldr = new GridHadoopClassLoader(null);
-
-                        jobCls = jobCls0 = ldr.loadClass(GridHadoopV2Job.class.getName());
-                    }
-                }
-            }
-
-            Constructor<?> constructor = jobCls0.getConstructor(GridHadoopJobId.class, GridHadoopDefaultJobInfo.class,
-                IgniteLogger.class);
-
-            return (GridHadoopJob)constructor.newInstance(jobId, this, log);
-        }
-        // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call.
-        catch (Throwable t) {
-            throw new IgniteCheckedException(t);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasCombiner() {
-        return hasCombiner;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean hasReducer() {
-        return reducers() > 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int reducers() {
-        return numReduces;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String jobName() {
-        return jobName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String user() {
-        return user;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeString(out, jobName);
-        U.writeString(out, user);
-
-        out.writeBoolean(hasCombiner);
-        out.writeInt(numReduces);
-
-        U.writeStringMap(out, props);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobName = U.readString(in);
-        user = U.readString(in);
-
-        hasCombiner = in.readBoolean();
-        numReduces = in.readInt();
-
-        props = U.readStringMap(in);
-    }
-
-    /**
-     * @return Properties of the job.
-     */
-    public Map<String, String> properties() {
-        return props;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
deleted file mode 100644
index 55e3690..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
+++ /dev/null
@@ -1,132 +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.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Hadoop facade implementation.
- */
-public class GridHadoopImpl implements GridHadoop {
-    /** Hadoop processor. */
-    private final IgniteHadoopProcessor proc;
-
-    /** Busy lock. */
-    private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
-
-    /**
-     * Constructor.
-     *
-     * @param proc Hadoop processor.
-     */
-    GridHadoopImpl(IgniteHadoopProcessor proc) {
-        this.proc = proc;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration configuration() {
-        return proc.config();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobId nextJobId() {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.nextJobId();
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get next job ID (grid is stopping).");
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.submit(jobId, jobInfo);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to submit job (grid is stopping).");
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.status(jobId);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get job status (grid is stopping).");
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.counters(jobId);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get job counters (grid is stopping).");
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.finishFuture(jobId);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to get job finish future (grid is stopping).");
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (busyLock.enterBusy()) {
-            try {
-                return proc.kill(jobId);
-            }
-            finally {
-                busyLock.leaveBusy();
-            }
-        }
-        else
-            throw new IllegalStateException("Failed to kill job (grid is stopping).");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSetup.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSetup.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSetup.java
deleted file mode 100644
index 66b1db4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopSetup.java
+++ /dev/null
@@ -1,505 +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.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.file.*;
-import java.text.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.IgniteVersionUtils.*;
-
-/**
- * Setup tool to configure Hadoop client.
- */
-public class GridHadoopSetup {
-    /** */
-    public static final String WINUTILS_EXE = "winutils.exe";
-
-    /** */
-    private static final FilenameFilter IGNITE_JARS = new FilenameFilter() {
-        @Override public boolean accept(File dir, String name) {
-            return name.startsWith("ignite-") && name.endsWith(".jar");
-        }
-    };
-
-    /**
-     * The main method.
-     * @param ignore Params.
-     */
-    public static void main(String[] ignore) {
-        X.println(
-            "   __________  ________________ ",
-            "  /  _/ ___/ |/ /  _/_  __/ __/ ",
-            " _/ // (_ /    // /  / / / _/   ",
-            "/___/\\___/_/|_/___/ /_/ /___/  ",
-            "                for Apache Hadoop        ",
-            "  ");
-
-        println("Version " + ACK_VER_STR);
-
-        configureHadoop();
-    }
-
-    /**
-     * This operation prepares the clean unpacked Hadoop distributive to work as client with Ignite-Hadoop.
-     * It performs these operations:
-     * <ul>
-     *     <li>Check for setting of HADOOP_HOME environment variable.</li>
-     *     <li>Try to resolve HADOOP_COMMON_HOME or evaluate it relative to HADOOP_HOME.</li>
-     *     <li>In Windows check if winutils.exe exists and try to fix issue with some restrictions.</li>
-     *     <li>In Windows check new line character issues in CMD scripts.</li>
-     *     <li>Scan Hadoop lib directory to detect Ignite JARs. If these don't exist tries to create ones.</li>
-     * </ul>
-     */
-    private static void configureHadoop() {
-        String igniteHome = U.getIgniteHome();
-
-        println("IGNITE_HOME is set to '" + igniteHome + "'.");
-
-        checkIgniteHome(igniteHome);
-
-        String homeVar = "HADOOP_HOME";
-        String hadoopHome = System.getenv(homeVar);
-
-        if (F.isEmpty(hadoopHome)) {
-            homeVar = "HADOOP_PREFIX";
-            hadoopHome = System.getenv(homeVar);
-        }
-
-        if (F.isEmpty(hadoopHome))
-            exit("Neither HADOOP_HOME nor HADOOP_PREFIX environment variable is set. Please set one of them to a " +
-                "valid Hadoop installation directory and run setup tool again.", null);
-
-        hadoopHome = hadoopHome.replaceAll("\"", "");
-
-        println(homeVar + " is set to '" + hadoopHome + "'.");
-
-        String hiveHome = System.getenv("HIVE_HOME");
-
-        if (!F.isEmpty(hiveHome)) {
-            hiveHome = hiveHome.replaceAll("\"", "");
-
-            println("HIVE_HOME is set to '" + hiveHome + "'.");
-        }
-
-        File hadoopDir = new File(hadoopHome);
-
-        if (!hadoopDir.exists())
-            exit("Hadoop installation folder does not exist.", null);
-
-        if (!hadoopDir.isDirectory())
-            exit("HADOOP_HOME must point to a directory.", null);
-
-        if (!hadoopDir.canRead())
-            exit("Hadoop installation folder can not be read. Please check permissions.", null);
-
-        File hadoopCommonDir;
-
-        String hadoopCommonHome = System.getenv("HADOOP_COMMON_HOME");
-
-        if (F.isEmpty(hadoopCommonHome)) {
-            hadoopCommonDir = new File(hadoopDir, "share/hadoop/common");
-
-            println("HADOOP_COMMON_HOME is not set, will use '" + hadoopCommonDir.getPath() + "'.");
-        }
-        else {
-            println("HADOOP_COMMON_HOME is set to '" + hadoopCommonHome + "'.");
-
-            hadoopCommonDir = new File(hadoopCommonHome);
-        }
-
-        if (!hadoopCommonDir.canRead())
-            exit("Failed to read Hadoop common dir in '" + hadoopCommonHome + "'.", null);
-
-        File hadoopCommonLibDir = new File(hadoopCommonDir, "lib");
-
-        if (!hadoopCommonLibDir.canRead())
-            exit("Failed to read Hadoop 'lib' folder in '" + hadoopCommonLibDir.getPath() + "'.", null);
-
-        if (U.isWindows()) {
-            checkJavaPathSpaces();
-
-            File hadoopBinDir = new File(hadoopDir, "bin");
-
-            if (!hadoopBinDir.canRead())
-                exit("Failed to read subdirectory 'bin' in HADOOP_HOME.", null);
-
-            File winutilsFile = new File(hadoopBinDir, WINUTILS_EXE);
-
-            if (!winutilsFile.exists()) {
-                if (ask("File '" + WINUTILS_EXE + "' does not exist. " +
-                    "It may be replaced by a stub. Create it?")) {
-                    println("Creating file stub '" + winutilsFile.getAbsolutePath() + "'.");
-
-                    boolean ok = false;
-
-                    try {
-                        ok = winutilsFile.createNewFile();
-                    }
-                    catch (IOException ignore) {
-                        // No-op.
-                    }
-
-                    if (!ok)
-                        exit("Failed to create '" + WINUTILS_EXE + "' file. Please check permissions.", null);
-                }
-                else
-                    println("Ok. But Hadoop client probably will not work on Windows this way...");
-            }
-
-            processCmdFiles(hadoopDir, "bin", "sbin", "libexec");
-        }
-
-        File igniteLibs = new File(new File(igniteHome), "libs");
-
-        if (!igniteLibs.exists())
-            exit("Ignite 'libs' folder is not found.", null);
-
-        Collection<File> jarFiles = new ArrayList<>();
-
-        addJarsInFolder(jarFiles, igniteLibs);
-        addJarsInFolder(jarFiles, new File(igniteLibs, "ignite-hadoop"));
-
-        boolean jarsLinksCorrect = true;
-
-        for (File file : jarFiles) {
-            File link = new File(hadoopCommonLibDir, file.getName());
-
-            jarsLinksCorrect &= isJarLinkCorrect(link, file);
-
-            if (!jarsLinksCorrect)
-                break;
-        }
-
-        if (!jarsLinksCorrect) {
-            if (ask("Ignite JAR files are not found in Hadoop 'lib' directory. " +
-                "Create appropriate symbolic links?")) {
-                File[] oldIgniteJarFiles = hadoopCommonLibDir.listFiles(IGNITE_JARS);
-
-                if (oldIgniteJarFiles.length > 0 && ask("The Hadoop 'lib' directory contains JARs from other Ignite " +
-                    "installation. They must be deleted to continue. Continue?")) {
-                    for (File file : oldIgniteJarFiles) {
-                        println("Deleting file '" + file.getAbsolutePath() + "'.");
-
-                        if (!file.delete())
-                            exit("Failed to delete file '" + file.getPath() + "'.", null);
-                    }
-                }
-
-                for (File file : jarFiles) {
-                    File targetFile = new File(hadoopCommonLibDir, file.getName());
-
-                    try {
-                        println("Creating symbolic link '" + targetFile.getAbsolutePath() + "'.");
-
-                        Files.createSymbolicLink(targetFile.toPath(), file.toPath());
-                    }
-                    catch (IOException e) {
-                        if (U.isWindows()) {
-                            warn("Ability to create symbolic links is required!");
-                            warn("On Windows platform you have to grant permission 'Create symbolic links'");
-                            warn("to your user or run the Accelerator as Administrator.");
-                        }
-
-                        exit("Creating symbolic link failed! Check permissions.", e);
-                    }
-                }
-            }
-            else
-                println("Ok. But Hadoop client will not be able to talk to Ignite cluster without those JARs in classpath...");
-        }
-
-        File hadoopEtc = new File(hadoopDir, "etc" + File.separator + "hadoop");
-
-        File igniteDocs = new File(igniteHome, "docs");
-
-        if (!igniteDocs.canRead())
-            exit("Failed to read Ignite 'docs' folder at '" + igniteDocs.getAbsolutePath() + "'.", null);
-
-        if (hadoopEtc.canWrite()) { // TODO Bigtop
-            if (ask("Replace 'core-site.xml' and 'mapred-site.xml' files with preconfigured templates " +
-                "(existing files will be backed up)?")) {
-                replaceWithBackup(new File(igniteDocs, "core-site.ignite.xml"), new File(hadoopEtc, "core-site.xml"));
-
-                replaceWithBackup(new File(igniteDocs, "mapred-site.ignite.xml"), new File(hadoopEtc, "mapred-site.xml"));
-            }
-            else
-                println("Ok. You can configure them later, the templates are available at Ignite's 'docs' directory...");
-        }
-
-        if (!F.isEmpty(hiveHome)) {
-            File hiveConfDir = new File(hiveHome + File.separator + "conf");
-
-            if (!hiveConfDir.canWrite())
-                warn("Can not write to '" + hiveConfDir.getAbsolutePath() + "'. To run Hive queries you have to " +
-                    "configure 'hive-site.xml' manually. The template is available at Ignite's 'docs' directory.");
-            else if (ask("Replace 'hive-site.xml' with preconfigured template (existing file will be backed up)?"))
-                replaceWithBackup(new File(igniteDocs, "hive-site.ignite.xml"), new File(hiveConfDir, "hive-site.xml"));
-            else
-                println("Ok. You can configure it later, the template is available at Ignite's 'docs' directory...");
-        }
-
-        println("Apache Hadoop setup is complete.");
-    }
-
-    /**
-     * @param jarFiles Jars.
-     * @param folder Folder.
-     */
-    private static void addJarsInFolder(Collection<File> jarFiles, File folder) {
-        if (!folder.exists())
-            exit("Folder '" + folder.getAbsolutePath() + "' is not found.", null);
-
-        jarFiles.addAll(Arrays.asList(folder.listFiles(IGNITE_JARS)));
-    }
-
-    /**
-     * Checks that JAVA_HOME does not contain space characters.
-     */
-    private static void checkJavaPathSpaces() {
-        String javaHome = System.getProperty("java.home");
-
-        if (javaHome.contains(" ")) {
-            warn("Java installation path contains space characters!");
-            warn("Hadoop client will not be able to start using '" + javaHome + "'.");
-            warn("Please install JRE to path which does not contain spaces and point JAVA_HOME to that installation.");
-        }
-    }
-
-    /**
-     * Checks Ignite home.
-     *
-     * @param igniteHome Ignite home.
-     */
-    private static void checkIgniteHome(String igniteHome) {
-        URL jarUrl = U.class.getProtectionDomain().getCodeSource().getLocation();
-
-        try {
-            Path jar = Paths.get(jarUrl.toURI());
-            Path igHome = Paths.get(igniteHome);
-
-            if (!jar.startsWith(igHome))
-                exit("Ignite JAR files are not under IGNITE_HOME.", null);
-        }
-        catch (Exception e) {
-            exit(e.getMessage(), e);
-        }
-    }
-
-    /**
-     * Replaces target file with source file.
-     *
-     * @param from From.
-     * @param to To.
-     */
-    private static void replaceWithBackup(File from, File to) {
-        if (!from.canRead())
-            exit("Failed to read source file '" + from.getAbsolutePath() + "'.", null);
-
-        println("Replacing file '" + to.getAbsolutePath() + "'.");
-
-        try {
-            U.copy(from, renameToBak(to), true);
-        }
-        catch (IOException e) {
-            exit("Failed to replace file '" + to.getAbsolutePath() + "'.", e);
-        }
-    }
-
-    /**
-     * Renames file for backup.
-     *
-     * @param file File.
-     * @return File.
-     */
-    private static File renameToBak(File file) {
-        DateFormat fmt = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss");
-
-        if (file.exists() && !file.renameTo(new File(file.getAbsolutePath() + "." + fmt.format(new Date()) + ".bak")))
-            exit("Failed to rename file '" + file.getPath() + "'.", null);
-
-        return file;
-    }
-
-    /**
-     * Checks if link is correct.
-     *
-     * @param link Symbolic link.
-     * @param correctTarget Correct link target.
-     * @return {@code true} If link target is correct.
-     */
-    private static boolean isJarLinkCorrect(File link, File correctTarget) {
-        if (!Files.isSymbolicLink(link.toPath()))
-            return false; // It is a real file or it does not exist.
-
-        Path target = null;
-
-        try {
-            target = Files.readSymbolicLink(link.toPath());
-        }
-        catch (IOException e) {
-            exit("Failed to read symbolic link: " + link.getAbsolutePath(), e);
-        }
-
-        return Files.exists(target) && target.toFile().equals(correctTarget);
-    }
-
-    /**
-     * Writes the question end read the boolean answer from the console.
-     *
-     * @param question Question to write.
-     * @return {@code true} if user inputs 'Y' or 'y', {@code false} otherwise.
-     */
-    private static boolean ask(String question) {
-        X.println();
-        X.print(" <  " + question + " (Y/N): ");
-
-        String answer = null;
-
-        if (!F.isEmpty(System.getenv("IGNITE_HADOOP_SETUP_YES")))
-            answer = "Y";
-        else {
-            BufferedReader br = new BufferedReader(new InputStreamReader(System.in));
-
-            try {
-                answer = br.readLine();
-            }
-            catch (IOException e) {
-                exit("Failed to read answer: " + e.getMessage(), e);
-            }
-        }
-
-        if (answer != null && "Y".equals(answer.toUpperCase().trim())) {
-            X.println(" >  Yes.");
-
-            return true;
-        }
-        else {
-            X.println(" >  No.");
-
-            return false;
-        }
-    }
-
-    /**
-     * Exit with message.
-     *
-     * @param msg Exit message.
-     */
-    private static void exit(String msg, Exception e) {
-        X.println("    ");
-        X.println("  # " + msg);
-        X.println("  # Setup failed, exiting... ");
-
-        if (e != null && !F.isEmpty(System.getenv("IGNITE_HADOOP_SETUP_DEBUG")))
-            e.printStackTrace();
-
-        System.exit(1);
-    }
-
-    /**
-     * Prints message.
-     *
-     * @param msg Message.
-     */
-    private static void println(String msg) {
-        X.println("  > " + msg);
-    }
-
-    /**
-     * Prints warning.
-     *
-     * @param msg Message.
-     */
-    private static void warn(String msg) {
-        X.println("  ! " + msg);
-    }
-
-    /**
-     * Checks that CMD files have valid MS Windows new line characters. If not, writes question to console and reads the
-     * answer. If it's 'Y' then backups original files and corrects invalid new line characters.
-     *
-     * @param rootDir Root directory to process.
-     * @param dirs Directories inside of the root to process.
-     */
-    private static void processCmdFiles(File rootDir, String... dirs) {
-        boolean answer = false;
-
-        for (String dir : dirs) {
-            File subDir = new File(rootDir, dir);
-
-            File[] cmdFiles = subDir.listFiles(new FilenameFilter() {
-                @Override public boolean accept(File dir, String name) {
-                    return name.toLowerCase().endsWith(".cmd");
-                }
-            });
-
-            for (File file : cmdFiles) {
-                String content = null;
-
-                try (Scanner scanner = new Scanner(file)) {
-                    content = scanner.useDelimiter("\\Z").next();
-                }
-                catch (FileNotFoundException e) {
-                    exit("Failed to read file '" + file + "'.", e);
-                }
-
-                boolean invalid = false;
-
-                for (int i = 0; i < content.length(); i++) {
-                    if (content.charAt(i) == '\n' && (i == 0 || content.charAt(i - 1) != '\r')) {
-                        invalid = true;
-
-                        break;
-                    }
-                }
-
-                if (invalid) {
-                    answer = answer || ask("One or more *.CMD files has invalid new line character. Replace them?");
-
-                    if (!answer) {
-                        println("Ok. But Windows most probably will fail to execute them...");
-
-                        return;
-                    }
-
-                    println("Fixing newline characters in file '" + file.getAbsolutePath() + "'.");
-
-                    renameToBak(file);
-
-                    try (BufferedWriter writer = new BufferedWriter(new FileWriter(file))) {
-                        for (int i = 0; i < content.length(); i++) {
-                            if (content.charAt(i) == '\n' && (i == 0 || content.charAt(i - 1) != '\r'))
-                                writer.write("\r");
-
-                            writer.write(content.charAt(i));
-                        }
-                    }
-                    catch (IOException e) {
-                        exit("Failed to write file '" + file.getPath() + "': " + e.getMessage(), e);
-                    }
-                }
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskCancelledException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskCancelledException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskCancelledException.java
deleted file mode 100644
index c762181..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskCancelledException.java
+++ /dev/null
@@ -1,35 +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.*;
-
-/**
- * Exception that throws when the task is cancelling.
- */
-public class GridHadoopTaskCancelledException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * @param msg Exception message.
-     */
-    public GridHadoopTaskCancelledException(String msg) {
-        super(msg);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopUtils.java
deleted file mode 100644
index 763f45a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopUtils.java
+++ /dev/null
@@ -1,308 +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.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.JobID;
-import org.apache.hadoop.mapreduce.JobPriority;
-import org.apache.hadoop.mapreduce.JobStatus;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Hadoop utility methods.
- */
-public class GridHadoopUtils {
-    /** Property to store timestamp of new job id request. */
-    public static final String REQ_NEW_JOBID_TS_PROPERTY = "ignite.job.requestNewIdTs";
-
-    /** Property to store timestamp of response of new job id request. */
-    public static final String RESPONSE_NEW_JOBID_TS_PROPERTY = "ignite.job.responseNewIdTs";
-
-    /** Property to store timestamp of job submission. */
-    public static final String JOB_SUBMISSION_START_TS_PROPERTY = "ignite.job.submissionStartTs";
-
-    /** Property to set custom writer of job statistics. */
-    public static final String JOB_COUNTER_WRITER_PROPERTY = "ignite.counters.writer";
-
-    /** Staging constant. */
-    private static final String STAGING_CONSTANT = ".staging";
-
-    /** Old mapper class attribute. */
-    private static final String OLD_MAP_CLASS_ATTR = "mapred.mapper.class";
-
-    /** Old reducer class attribute. */
-    private static final String OLD_REDUCE_CLASS_ATTR = "mapred.reducer.class";
-
-    /**
-     * Wraps native split.
-     *
-     * @param id Split ID.
-     * @param split Split.
-     * @param hosts Hosts.
-     * @throws IOException If failed.
-     */
-    public static GridHadoopSplitWrapper wrapSplit(int id, Object split, String[] hosts) throws IOException {
-        ByteArrayOutputStream arr = new ByteArrayOutputStream();
-        ObjectOutput out = new ObjectOutputStream(arr);
-
-        assert split instanceof Writable;
-
-        ((Writable)split).write(out);
-
-        out.flush();
-
-        return new GridHadoopSplitWrapper(id, split.getClass().getName(), arr.toByteArray(), hosts);
-    }
-
-    /**
-     * Unwraps native split.
-     *
-     * @param o Wrapper.
-     * @return Split.
-     */
-    public static Object unwrapSplit(GridHadoopSplitWrapper o) {
-        try {
-            Writable w = (Writable)GridHadoopUtils.class.getClassLoader().loadClass(o.className()).newInstance();
-
-            w.readFields(new ObjectInputStream(new ByteArrayInputStream(o.bytes())));
-
-            return w;
-        }
-        catch (Exception e) {
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Convert Ignite job status to Hadoop job status.
-     *
-     * @param status Ignite job status.
-     * @return Hadoop job status.
-     */
-    public static JobStatus status(GridHadoopJobStatus status, Configuration conf) {
-        JobID jobId = new JobID(status.jobId().globalId().toString(), status.jobId().localId());
-
-        float setupProgress = 0;
-        float mapProgress = 0;
-        float reduceProgress = 0;
-        float cleanupProgress = 0;
-
-        JobStatus.State state = JobStatus.State.RUNNING;
-
-        switch (status.jobPhase()) {
-            case PHASE_SETUP:
-                setupProgress = 0.42f;
-
-                break;
-
-            case PHASE_MAP:
-                setupProgress = 1;
-                mapProgress = 1f - status.pendingMapperCnt() / (float)status.totalMapperCnt();
-
-                break;
-
-            case PHASE_REDUCE:
-                assert status.totalReducerCnt() > 0;
-
-                setupProgress = 1;
-                mapProgress = 1;
-                reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
-
-                break;
-
-            case PHASE_CANCELLING:
-            case PHASE_COMPLETE:
-                if (!status.isFailed()) {
-                    setupProgress = 1;
-                    mapProgress = 1;
-                    reduceProgress = 1;
-                    cleanupProgress = 1;
-
-                    state = JobStatus.State.SUCCEEDED;
-                }
-                else
-                    state = JobStatus.State.FAILED;
-
-                break;
-
-            default:
-                assert false;
-        }
-
-        return new JobStatus(jobId, setupProgress, mapProgress, reduceProgress, cleanupProgress, state,
-            JobPriority.NORMAL, status.user(), status.jobName(), jobFile(conf, status.user(), jobId).toString(), "N/A");
-    }
-
-    /**
-     * Gets staging area directory.
-     *
-     * @param conf Configuration.
-     * @param usr User.
-     * @return Staging area directory.
-     */
-    public static Path stagingAreaDir(Configuration conf, String usr) {
-        return new Path(conf.get(MRJobConfig.MR_AM_STAGING_DIR, MRJobConfig.DEFAULT_MR_AM_STAGING_DIR)
-            + Path.SEPARATOR + usr + Path.SEPARATOR + STAGING_CONSTANT);
-    }
-
-    /**
-     * Gets job file.
-     *
-     * @param conf Configuration.
-     * @param usr User.
-     * @param jobId Job ID.
-     * @return Job file.
-     */
-    public static Path jobFile(Configuration conf, String usr, JobID jobId) {
-        return new Path(stagingAreaDir(conf, usr), jobId.toString() + Path.SEPARATOR + MRJobConfig.JOB_CONF_FILE);
-    }
-
-    /**
-     * Checks the attribute in configuration is not set.
-     *
-     * @param attr Attribute name.
-     * @param msg Message for creation of exception.
-     * @throws IgniteCheckedException If attribute is set.
-     */
-    public static void ensureNotSet(Configuration cfg, String attr, String msg) throws IgniteCheckedException {
-        if (cfg.get(attr) != null)
-            throw new IgniteCheckedException(attr + " is incompatible with " + msg + " mode.");
-    }
-
-    /**
-     * Creates JobInfo from hadoop configuration.
-     *
-     * @param cfg Hadoop configuration.
-     * @return Job info.
-     * @throws IgniteCheckedException If failed.
-     */
-    public static GridHadoopDefaultJobInfo createJobInfo(Configuration cfg) throws IgniteCheckedException {
-        JobConf jobConf = new JobConf(cfg);
-
-        boolean hasCombiner = jobConf.get("mapred.combiner.class") != null
-                || jobConf.get(MRJobConfig.COMBINE_CLASS_ATTR) != null;
-
-        int numReduces = jobConf.getNumReduceTasks();
-
-        jobConf.setBooleanIfUnset("mapred.mapper.new-api", jobConf.get(OLD_MAP_CLASS_ATTR) == null);
-
-        if (jobConf.getUseNewMapper()) {
-            String mode = "new map API";
-
-            ensureNotSet(jobConf, "mapred.input.format.class", mode);
-            ensureNotSet(jobConf, OLD_MAP_CLASS_ATTR, mode);
-
-            if (numReduces != 0)
-                ensureNotSet(jobConf, "mapred.partitioner.class", mode);
-            else
-                ensureNotSet(jobConf, "mapred.output.format.class", mode);
-        }
-        else {
-            String mode = "map compatibility";
-
-            ensureNotSet(jobConf, MRJobConfig.INPUT_FORMAT_CLASS_ATTR, mode);
-            ensureNotSet(jobConf, MRJobConfig.MAP_CLASS_ATTR, mode);
-
-            if (numReduces != 0)
-                ensureNotSet(jobConf, MRJobConfig.PARTITIONER_CLASS_ATTR, mode);
-            else
-                ensureNotSet(jobConf, MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, mode);
-        }
-
-        if (numReduces != 0) {
-            jobConf.setBooleanIfUnset("mapred.reducer.new-api", jobConf.get(OLD_REDUCE_CLASS_ATTR) == null);
-
-            if (jobConf.getUseNewReducer()) {
-                String mode = "new reduce API";
-
-                ensureNotSet(jobConf, "mapred.output.format.class", mode);
-                ensureNotSet(jobConf, OLD_REDUCE_CLASS_ATTR, mode);
-            }
-            else {
-                String mode = "reduce compatibility";
-
-                ensureNotSet(jobConf, MRJobConfig.OUTPUT_FORMAT_CLASS_ATTR, mode);
-                ensureNotSet(jobConf, MRJobConfig.REDUCE_CLASS_ATTR, mode);
-            }
-        }
-
-        Map<String, String> props = new HashMap<>();
-
-        for (Map.Entry<String, String> entry : jobConf)
-            props.put(entry.getKey(), entry.getValue());
-
-        return new GridHadoopDefaultJobInfo(jobConf.getJobName(), jobConf.getUser(), hasCombiner, numReduces, props);
-    }
-
-    /**
-     * Throws new {@link IgniteCheckedException} with original exception is serialized into string.
-     * This is needed to transfer error outside the current class loader.
-     *
-     * @param e Original exception.
-     * @return IgniteCheckedException New exception.
-     */
-    public static IgniteCheckedException transformException(Throwable e) {
-        ByteArrayOutputStream os = new ByteArrayOutputStream();
-
-        e.printStackTrace(new PrintStream(os, true));
-
-        return new IgniteCheckedException(os.toString());
-    }
-
-    /**
-     * Returns work directory for job execution.
-     *
-     * @param locNodeId Local node ID.
-     * @param jobId Job ID.
-     * @return Working directory for job.
-     * @throws IgniteCheckedException If Failed.
-     */
-    public static File jobLocalDir(UUID locNodeId, GridHadoopJobId jobId) throws IgniteCheckedException {
-        return new File(new File(U.resolveWorkDirectory("hadoop", false), "node-" + locNodeId), "job_" + jobId);
-    }
-
-    /**
-     * Returns subdirectory of job working directory for task execution.
-     *
-     * @param locNodeId Local node ID.
-     * @param info Task info.
-     * @return Working directory for task.
-     * @throws IgniteCheckedException If Failed.
-     */
-    public static File taskLocalDir(UUID locNodeId, GridHadoopTaskInfo info) throws IgniteCheckedException {
-        File jobLocDir = jobLocalDir(locNodeId, info.jobId());
-
-        return new File(jobLocDir, info.type() + "_" + info.taskNumber() + "_" + info.attempt());
-    }
-
-    /**
-     * Constructor.
-     */
-    private GridHadoopUtils() {
-        // No-op.
-    }
-}


[10/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
new file mode 100644
index 0000000..07bae6b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
@@ -0,0 +1,726 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Skip list.
+ */
+public class HadoopSkipList extends HadoopMultimapBase {
+    /** */
+    private static final int HEADS_SIZE = 24 + 33 * 8; // Offset + max level is from 0 to 32 inclusive.
+
+    /** Top level. */
+    private final AtomicInteger topLevel = new AtomicInteger(-1);
+
+    /** Heads for all the lists. */
+    private final long heads;
+
+    /** */
+    private final AtomicBoolean visitGuard = new AtomicBoolean();
+
+    /**
+     * @param jobInfo Job info.
+     * @param mem Memory.
+     */
+    public HadoopSkipList(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
+        super(jobInfo, mem);
+
+        heads = mem.allocate(HEADS_SIZE, true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        super.close();
+
+        mem.release(heads, HEADS_SIZE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
+        if (!visitGuard.compareAndSet(false, true))
+            return false;
+
+        for (long meta = nextMeta(heads, 0); meta != 0L; meta = nextMeta(meta, 0)) {
+            long valPtr = value(meta);
+
+            long lastVisited = ignoreLastVisited ? 0 : lastVisitedValue(meta);
+
+            if (valPtr != lastVisited) {
+                long k = key(meta);
+
+                v.onKey(k + 4, keySize(k));
+
+                lastVisitedValue(meta, valPtr); // Set it to the first value in chain.
+
+                do {
+                    v.onValue(valPtr + 12, valueSize(valPtr));
+
+                    valPtr = nextValue(valPtr);
+                }
+                while (valPtr != lastVisited);
+            }
+        }
+
+        visitGuard.lazySet(false);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+        return new AdderImpl(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        Input in = new Input(taskCtx);
+
+        Comparator<Object> grpCmp = taskCtx.groupComparator();
+
+        if (grpCmp != null)
+            return new GroupedInput(grpCmp, in);
+
+        return in;
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Key pointer.
+     */
+    private long key(long meta) {
+        return mem.readLong(meta);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param key Key pointer.
+     */
+    private void key(long meta, long key) {
+        mem.writeLong(meta, key);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Value pointer.
+     */
+    private long value(long meta) {
+        return mem.readLongVolatile(meta + 8);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param valPtr Value pointer.
+     */
+    private void value(long meta, long valPtr) {
+        mem.writeLongVolatile(meta + 8, valPtr);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param oldValPtr Old first value pointer.
+     * @param newValPtr New first value pointer.
+     * @return {@code true} If operation succeeded.
+     */
+    private boolean casValue(long meta, long oldValPtr, long newValPtr) {
+        return mem.casLong(meta + 8, oldValPtr, newValPtr);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Last visited value pointer.
+     */
+    private long lastVisitedValue(long meta) {
+        return mem.readLong(meta + 16);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param valPtr Last visited value pointer.
+     */
+    private void lastVisitedValue(long meta, long valPtr) {
+        mem.writeLong(meta + 16, valPtr);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param level Level.
+     * @return Next meta pointer.
+     */
+    private long nextMeta(long meta, int level) {
+        assert meta > 0 : meta;
+
+        return mem.readLongVolatile(meta + 24 + 8 * level);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param level Level.
+     * @param oldNext Old next meta pointer.
+     * @param newNext New next meta pointer.
+     * @return {@code true} If operation succeeded.
+     */
+    private boolean casNextMeta(long meta, int level, long oldNext, long newNext) {
+        assert meta > 0 : meta;
+
+        return mem.casLong(meta + 24 + 8 * level, oldNext, newNext);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param level Level.
+     * @param nextMeta Next meta.
+     */
+    private void nextMeta(long meta, int level, long nextMeta) {
+        assert meta != 0;
+
+        mem.writeLong(meta + 24 + 8 * level, nextMeta);
+    }
+
+    /**
+     * @param keyPtr Key pointer.
+     * @return Key size.
+     */
+    private int keySize(long keyPtr) {
+        return mem.readInt(keyPtr);
+    }
+
+    /**
+     * @param keyPtr Key pointer.
+     * @param keySize Key size.
+     */
+    private void keySize(long keyPtr, int keySize) {
+        mem.writeInt(keyPtr, keySize);
+    }
+
+    /**
+     * @param rnd Random.
+     * @return Next level.
+     */
+    public static int randomLevel(Random rnd) {
+        int x = rnd.nextInt();
+
+        int level = 0;
+
+        while ((x & 1) != 0) { // Count sequential 1 bits.
+            level++;
+
+            x >>>= 1;
+        }
+
+        return level;
+    }
+
+    /**
+     * Reader.
+     */
+    private class Reader extends ReaderBase {
+        /**
+         * @param ser Serialization.
+         */
+        protected Reader(GridHadoopSerialization ser) {
+            super(ser);
+        }
+
+        /**
+         * @param meta Meta pointer.
+         * @return Key.
+         */
+        public Object readKey(long meta) {
+            assert meta > 0 : meta;
+
+            long k = key(meta);
+
+            try {
+                return read(k + 4, keySize(k));
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+    }
+
+    /**
+     * Adder.
+     */
+    private class AdderImpl extends AdderBase {
+        /** */
+        private final Comparator<Object> cmp;
+
+        /** */
+        private final Random rnd = new GridRandom();
+
+        /** */
+        private final GridLongList stack = new GridLongList(16);
+
+        /** */
+        private final Reader keyReader;
+
+        /**
+         * @param ctx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+            super(ctx);
+
+            keyReader = new Reader(keySer);
+
+            cmp = ctx.sortComparator();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object key, Object val) throws IgniteCheckedException {
+            A.notNull(val, "val");
+
+            add(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
+            KeyImpl k = reuse == null ? new KeyImpl() : (KeyImpl)reuse;
+
+            k.tmpKey = keySer.read(in, k.tmpKey);
+
+            k.meta = add(k.tmpKey, null);
+
+            return k;
+        }
+
+        /**
+         * @param key Key.
+         * @param val Value.
+         * @param level Level.
+         * @return Meta pointer.
+         */
+        private long createMeta(long key, long val, int level) {
+            int size = 32 + 8 * level;
+
+            long meta = allocate(size);
+
+            key(meta, key);
+            value(meta, val);
+            lastVisitedValue(meta, 0L);
+
+            for (int i = 32; i < size; i += 8) // Fill with 0.
+                mem.writeLong(meta + i, 0L);
+
+            return meta;
+        }
+
+        /**
+         * @param key Key.
+         * @return Pointer.
+         * @throws IgniteCheckedException If failed.
+         */
+        private long writeKey(Object key) throws IgniteCheckedException {
+            long keyPtr = write(4, key, keySer);
+            int keySize = writtenSize() - 4;
+
+            keySize(keyPtr, keySize);
+
+            return keyPtr;
+        }
+
+        /**
+         * @param prevMeta Previous meta.
+         * @param meta Next meta.
+         */
+        private void stackPush(long prevMeta, long meta) {
+            stack.add(prevMeta);
+            stack.add(meta);
+        }
+
+        /**
+         * Drops last remembered frame from the stack.
+         */
+        private void stackPop() {
+            stack.pop(2);
+        }
+
+        /**
+         * @param key Key.
+         * @param val Value.
+         * @return Meta pointer.
+         * @throws IgniteCheckedException If failed.
+         */
+        private long add(Object key, @Nullable Object val) throws IgniteCheckedException {
+            assert key != null;
+
+            stack.clear();
+
+            long valPtr = 0;
+
+            if (val != null) { // Write value.
+                valPtr = write(12, val, valSer);
+                int valSize = writtenSize() - 12;
+
+                nextValue(valPtr, 0);
+                valueSize(valPtr, valSize);
+            }
+
+            long keyPtr = 0;
+            long newMeta = 0;
+            int newMetaLevel = -1;
+
+            long prevMeta = heads;
+            int level = topLevel.get();
+            long meta = level < 0 ? 0 : nextMeta(heads, level);
+
+            for (;;) {
+                if (level < 0) { // We did not find our key, trying to add new meta.
+                    if (keyPtr == 0) { // Write key and create meta only once.
+                        keyPtr = writeKey(key);
+
+                        newMetaLevel = randomLevel(rnd);
+                        newMeta = createMeta(keyPtr, valPtr, newMetaLevel);
+                    }
+
+                    nextMeta(newMeta, 0, meta); // Set next to new meta before publishing.
+
+                    if (casNextMeta(prevMeta, 0, meta, newMeta)) { // New key was added successfully.
+                        laceUp(key, newMeta, newMetaLevel);
+
+                        return newMeta;
+                    }
+                    else { // Add failed, need to check out what was added by another thread.
+                        meta = nextMeta(prevMeta, level = 0);
+
+                        stackPop();
+                    }
+                }
+
+                int cmpRes = cmp(key, meta);
+
+                if (cmpRes == 0) { // Key found.
+                    if (newMeta != 0)  // Deallocate if we've allocated something.
+                        localDeallocate(keyPtr);
+
+                    if (valPtr == 0) // Only key needs to be added.
+                        return meta;
+
+                    for (;;) { // Add value for the key found.
+                        long nextVal = value(meta);
+
+                        nextValue(valPtr, nextVal);
+
+                        if (casValue(meta, nextVal, valPtr))
+                            return meta;
+                    }
+                }
+
+                assert cmpRes != 0;
+
+                if (cmpRes > 0) { // Go right.
+                    prevMeta = meta;
+                    meta = nextMeta(meta, level);
+
+                    if (meta != 0) // If nothing to the right then go down.
+                        continue;
+                }
+
+                while (--level >= 0) { // Go down.
+                    stackPush(prevMeta, meta); // Remember the path.
+
+                    long nextMeta = nextMeta(prevMeta, level);
+
+                    if (nextMeta != meta) { // If the meta is the same as on upper level go deeper.
+                        meta = nextMeta;
+
+                        assert meta != 0;
+
+                        break;
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param key Key.
+         * @param meta Meta pointer.
+         * @return Comparison result.
+         */
+        @SuppressWarnings("unchecked")
+        private int cmp(Object key, long meta) {
+            assert meta != 0;
+
+            return cmp.compare(key, keyReader.readKey(meta));
+        }
+
+        /**
+         * Adds appropriate index links between metas.
+         *
+         * @param newMeta Just added meta.
+         * @param newMetaLevel New level.
+         */
+        private void laceUp(Object key, long newMeta, int newMetaLevel) {
+            for (int level = 1; level <= newMetaLevel; level++) { // Go from the bottom up.
+                long prevMeta = heads;
+                long meta = 0;
+
+                if (!stack.isEmpty()) { // Get the path back.
+                    meta = stack.remove();
+                    prevMeta = stack.remove();
+                }
+
+                for (;;) {
+                    nextMeta(newMeta, level, meta);
+
+                    if (casNextMeta(prevMeta, level, meta, newMeta))
+                        break;
+
+                    long oldMeta = meta;
+
+                    meta = nextMeta(prevMeta, level); // Reread meta.
+
+                    for (;;) {
+                        int cmpRes = cmp(key, meta);
+
+                        if (cmpRes > 0) { // Go right.
+                            prevMeta = meta;
+                            meta = nextMeta(prevMeta, level);
+
+                            if (meta != oldMeta) // Old meta already known to be greater than ours or is 0.
+                                continue;
+                        }
+
+                        assert cmpRes != 0; // Two different metas with equal keys must be impossible.
+
+                        break; // Retry cas.
+                    }
+                }
+            }
+
+            if (!stack.isEmpty())
+                return; // Our level already lower than top.
+
+            for (;;) { // Raise top level.
+                int top = topLevel.get();
+
+                if (newMetaLevel <= top || topLevel.compareAndSet(top, newMetaLevel))
+                    break;
+            }
+        }
+
+        /**
+         * Key.
+         */
+        private class KeyImpl implements Key {
+            /** */
+            private long meta;
+
+            /** */
+            private Object tmpKey;
+
+            /**
+             * @return Meta pointer for the key.
+             */
+            public long address() {
+                return meta;
+            }
+
+            /**
+             * @param val Value.
+             */
+            @Override public void add(Value val) {
+                int size = val.size();
+
+                long valPtr = allocate(size + 12);
+
+                val.copyTo(valPtr + 12);
+
+                valueSize(valPtr, size);
+
+                long nextVal;
+
+                do {
+                    nextVal = value(meta);
+
+                    nextValue(valPtr, nextVal);
+                }
+                while(!casValue(meta, nextVal, valPtr));
+            }
+        }
+    }
+
+    /**
+     * Task input.
+     */
+    private class Input implements GridHadoopTaskInput {
+        /** */
+        private long metaPtr = heads;
+
+        /** */
+        private final Reader keyReader;
+
+        /** */
+        private final Reader valReader;
+
+        /**
+         * @param taskCtx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        private Input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+            keyReader = new Reader(taskCtx.keySerialization());
+            valReader = new Reader(taskCtx.valueSerialization());
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            metaPtr = nextMeta(metaPtr, 0);
+
+            return metaPtr != 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object key() {
+            return keyReader.readKey(metaPtr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<?> values() {
+            return new ValueIterator(value(metaPtr), valReader);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            keyReader.close();
+            valReader.close();
+        }
+    }
+
+    /**
+     * Grouped input using grouping comparator.
+     */
+    private class GroupedInput implements GridHadoopTaskInput {
+        /** */
+        private final Comparator<Object> grpCmp;
+
+        /** */
+        private final Input in;
+
+        /** */
+        private Object prevKey;
+
+        /** */
+        private Object nextKey;
+
+        /** */
+        private final GridLongList vals = new GridLongList();
+
+        /**
+         * @param grpCmp Grouping comparator.
+         * @param in Input.
+         */
+        private GroupedInput(Comparator<Object> grpCmp, Input in) {
+            this.grpCmp = grpCmp;
+            this.in = in;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            if (prevKey == null) { // First call.
+                if (!in.next())
+                    return false;
+
+                prevKey = in.key();
+
+                assert prevKey != null;
+
+                in.keyReader.resetReusedObject(null); // We need 2 instances of key object for comparison.
+
+                vals.add(value(in.metaPtr));
+            }
+            else {
+                if (in.metaPtr == 0) // We reached the end of the input.
+                    return false;
+
+                vals.clear();
+
+                vals.add(value(in.metaPtr));
+
+                in.keyReader.resetReusedObject(prevKey); // Switch key instances.
+
+                prevKey = nextKey;
+            }
+
+            while (in.next()) { // Fill with head value pointers with equal keys.
+                if (grpCmp.compare(prevKey, nextKey = in.key()) == 0)
+                    vals.add(value(in.metaPtr));
+                else
+                    break;
+            }
+
+            assert !vals.isEmpty();
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object key() {
+            return prevKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<?> values() {
+            assert !vals.isEmpty();
+
+            final ValueIterator valIter = new ValueIterator(vals.get(0), in.valReader);
+
+            return new Iterator<Object>() {
+                /** */
+                private int idx;
+
+                @Override public boolean hasNext() {
+                    if (!valIter.hasNext()) {
+                        if (++idx == vals.size())
+                            return false;
+
+                        valIter.head(vals.get(idx));
+
+                        assert valIter.hasNext();
+                    }
+
+                    return true;
+                }
+
+                @Override public Object next() {
+                    return valIter.next();
+                }
+
+                @Override public void remove() {
+                    valIter.remove();
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            in.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataInStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataInStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataInStream.java
deleted file mode 100644
index 8b4f0c4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataInStream.java
+++ /dev/null
@@ -1,170 +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.shuffle.streams;
-
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-
-import java.io.*;
-import java.nio.charset.*;
-
-/**
- * Data input stream.
- */
-public class GridHadoopDataInStream extends InputStream implements DataInput {
-    /** */
-    private final GridHadoopOffheapBuffer buf = new GridHadoopOffheapBuffer(0, 0);
-
-    /** */
-    private final GridUnsafeMemory mem;
-
-    /**
-     * @param mem Memory.
-     */
-    public GridHadoopDataInStream(GridUnsafeMemory mem) {
-        assert mem != null;
-
-        this.mem = mem;
-    }
-
-    /**
-     * @return Buffer.
-     */
-    public GridHadoopOffheapBuffer buffer() {
-        return buf;
-    }
-
-    /**
-     * @param size Size.
-     * @return Old pointer.
-     */
-    protected long move(long size) throws IOException {
-        long ptr = buf.move(size);
-
-        assert ptr != 0;
-
-        return ptr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read() throws IOException {
-        return readUnsignedByte();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read(byte[] b, int off, int len) throws IOException {
-        readFully(b, off, len);
-
-        return len;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long skip(long n) throws IOException {
-        move(n);
-
-        return n;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(byte[] b) throws IOException {
-        readFully(b, 0, b.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(byte[] b, int off, int len) throws IOException {
-        mem.readBytes(move(len), b, off, len);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int skipBytes(int n) throws IOException {
-        move(n);
-
-        return n;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readBoolean() throws IOException {
-        byte res = readByte();
-
-        if (res == 1)
-            return true;
-
-        assert res == 0 : res;
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte readByte() throws IOException {
-        return mem.readByte(move(1));
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readUnsignedByte() throws IOException {
-        return readByte() & 0xff;
-    }
-
-    /** {@inheritDoc} */
-    @Override public short readShort() throws IOException {
-        return mem.readShort(move(2));
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readUnsignedShort() throws IOException {
-        return readShort() & 0xffff;
-    }
-
-    /** {@inheritDoc} */
-    @Override public char readChar() throws IOException {
-        return (char)readShort();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int readInt() throws IOException {
-        return mem.readInt(move(4));
-    }
-
-    /** {@inheritDoc} */
-    @Override public long readLong() throws IOException {
-        return mem.readLong(move(8));
-    }
-
-    /** {@inheritDoc} */
-    @Override public float readFloat() throws IOException {
-        return mem.readFloat(move(4));
-    }
-
-    /** {@inheritDoc} */
-    @Override public double readDouble() throws IOException {
-        return mem.readDouble(move(8));
-    }
-
-    /** {@inheritDoc} */
-    @Override public String readLine() throws IOException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String readUTF() throws IOException {
-        byte[] bytes = new byte[readInt()];
-
-        if (bytes.length != 0)
-            readFully(bytes);
-
-        return new String(bytes, StandardCharsets.UTF_8);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataOutStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataOutStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataOutStream.java
deleted file mode 100644
index 8b837c8..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataOutStream.java
+++ /dev/null
@@ -1,131 +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.shuffle.streams;
-
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-
-import java.io.*;
-import java.nio.charset.*;
-
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
-
-/**
- * Data output stream.
- */
-public class GridHadoopDataOutStream extends OutputStream implements DataOutput {
-    /** */
-    private final GridHadoopOffheapBuffer buf = new GridHadoopOffheapBuffer(0, 0);
-
-    /** */
-    private final GridUnsafeMemory mem;
-
-    /**
-     * @param mem Memory.
-     */
-    public GridHadoopDataOutStream(GridUnsafeMemory mem) {
-        this.mem = mem;
-    }
-
-    /**
-     * @return Buffer.
-     */
-    public GridHadoopOffheapBuffer buffer() {
-        return buf;
-    }
-
-    /**
-     * @param size Size.
-     * @return Old pointer or {@code 0} if move was impossible.
-     */
-    public long move(long size) {
-        return buf.move(size);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(int b) {
-        writeByte(b);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(byte[] b) {
-        write(b, 0, b.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(byte[] b, int off, int len) {
-        UNSAFE.copyMemory(b, BYTE_ARR_OFF + off, null, move(len), len);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBoolean(boolean v) {
-        writeByte(v ? 1 : 0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeByte(int v) {
-        mem.writeByte(move(1), (byte)v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeShort(int v) {
-        mem.writeShort(move(2), (short)v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChar(int v) {
-        writeShort(v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeInt(int v) {
-        mem.writeInt(move(4), v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeLong(long v) {
-        mem.writeLong(move(8), v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeFloat(float v) {
-        mem.writeFloat(move(4), v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeDouble(double v) {
-        mem.writeDouble(move(8), v);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeBytes(String s) {
-        writeUTF(s);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeChars(String s) {
-        writeUTF(s);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeUTF(String s) {
-        byte[] b = s.getBytes(StandardCharsets.UTF_8);
-
-        writeInt(b.length);
-        write(b);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopOffheapBuffer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopOffheapBuffer.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopOffheapBuffer.java
deleted file mode 100644
index f9f0e1d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopOffheapBuffer.java
+++ /dev/null
@@ -1,122 +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.shuffle.streams;
-
-/**
- * Offheap buffer.
- */
-public class GridHadoopOffheapBuffer {
-    /** Buffer begin address. */
-    private long bufPtr;
-
-    /** The first address we do not own. */
-    private long bufEnd;
-
-    /** Current read or write pointer. */
-    private long posPtr;
-
-    /**
-     * @param bufPtr Pointer to buffer begin.
-     * @param bufSize Size of the buffer.
-     */
-    public GridHadoopOffheapBuffer(long bufPtr, long bufSize) {
-        set(bufPtr, bufSize);
-    }
-
-    /**
-     * @param bufPtr Pointer to buffer begin.
-     * @param bufSize Size of the buffer.
-     */
-    public void set(long bufPtr, long bufSize) {
-        this.bufPtr = bufPtr;
-
-        posPtr = bufPtr;
-        bufEnd = bufPtr + bufSize;
-    }
-
-    /**
-     * @return Pointer to internal buffer begin.
-     */
-    public long begin() {
-        return bufPtr;
-    }
-
-    /**
-     * @return Buffer capacity.
-     */
-    public long capacity() {
-        return bufEnd - bufPtr;
-    }
-
-    /**
-     * @return Remaining capacity.
-     */
-    public long remaining() {
-        return bufEnd - posPtr;
-    }
-
-    /**
-     * @return Absolute pointer to the current position inside of the buffer.
-     */
-    public long pointer() {
-        return posPtr;
-    }
-
-    /**
-     * @param ptr Absolute pointer to the current position inside of the buffer.
-     */
-    public void pointer(long ptr) {
-        assert ptr >= bufPtr : bufPtr + " <= " + ptr;
-        assert ptr <= bufEnd : bufEnd + " <= " + bufPtr;
-
-        posPtr = ptr;
-    }
-
-    /**
-     * @param size Size move on.
-     * @return Old position pointer or {@code 0} if move goes beyond the end of the buffer.
-     */
-    public long move(long size) {
-        assert size > 0 : size;
-
-        long oldPos = posPtr;
-        long newPos = oldPos + size;
-
-        if (newPos > bufEnd)
-            return 0;
-
-        posPtr = newPos;
-
-        return oldPos;
-    }
-
-    /**
-     * @param ptr Pointer.
-     * @return {@code true} If the given pointer is inside of this buffer.
-     */
-    public boolean isInside(long ptr) {
-        return ptr >= bufPtr && ptr <= bufEnd;
-    }
-
-    /**
-     * Resets position to the beginning of buffer.
-     */
-    public void reset() {
-        posPtr = bufPtr;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataInStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataInStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataInStream.java
new file mode 100644
index 0000000..8a1ee70
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataInStream.java
@@ -0,0 +1,170 @@
+/*
+ * 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.shuffle.streams;
+
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+
+import java.io.*;
+import java.nio.charset.*;
+
+/**
+ * Data input stream.
+ */
+public class HadoopDataInStream extends InputStream implements DataInput {
+    /** */
+    private final HadoopOffheapBuffer buf = new HadoopOffheapBuffer(0, 0);
+
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /**
+     * @param mem Memory.
+     */
+    public HadoopDataInStream(GridUnsafeMemory mem) {
+        assert mem != null;
+
+        this.mem = mem;
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public HadoopOffheapBuffer buffer() {
+        return buf;
+    }
+
+    /**
+     * @param size Size.
+     * @return Old pointer.
+     */
+    protected long move(long size) throws IOException {
+        long ptr = buf.move(size);
+
+        assert ptr != 0;
+
+        return ptr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read() throws IOException {
+        return readUnsignedByte();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read(byte[] b, int off, int len) throws IOException {
+        readFully(b, off, len);
+
+        return len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long skip(long n) throws IOException {
+        move(n);
+
+        return n;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(byte[] b) throws IOException {
+        readFully(b, 0, b.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(byte[] b, int off, int len) throws IOException {
+        mem.readBytes(move(len), b, off, len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int skipBytes(int n) throws IOException {
+        move(n);
+
+        return n;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean() throws IOException {
+        byte res = readByte();
+
+        if (res == 1)
+            return true;
+
+        assert res == 0 : res;
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte readByte() throws IOException {
+        return mem.readByte(move(1));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedByte() throws IOException {
+        return readByte() & 0xff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort() throws IOException {
+        return mem.readShort(move(2));
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedShort() throws IOException {
+        return readShort() & 0xffff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar() throws IOException {
+        return (char)readShort();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt() throws IOException {
+        return mem.readInt(move(4));
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong() throws IOException {
+        return mem.readLong(move(8));
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat() throws IOException {
+        return mem.readFloat(move(4));
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble() throws IOException {
+        return mem.readDouble(move(8));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String readLine() throws IOException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String readUTF() throws IOException {
+        byte[] bytes = new byte[readInt()];
+
+        if (bytes.length != 0)
+            readFully(bytes);
+
+        return new String(bytes, StandardCharsets.UTF_8);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataOutStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataOutStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataOutStream.java
new file mode 100644
index 0000000..51bddf9
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopDataOutStream.java
@@ -0,0 +1,131 @@
+/*
+ * 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.shuffle.streams;
+
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+
+import java.io.*;
+import java.nio.charset.*;
+
+import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
+
+/**
+ * Data output stream.
+ */
+public class HadoopDataOutStream extends OutputStream implements DataOutput {
+    /** */
+    private final HadoopOffheapBuffer buf = new HadoopOffheapBuffer(0, 0);
+
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /**
+     * @param mem Memory.
+     */
+    public HadoopDataOutStream(GridUnsafeMemory mem) {
+        this.mem = mem;
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public HadoopOffheapBuffer buffer() {
+        return buf;
+    }
+
+    /**
+     * @param size Size.
+     * @return Old pointer or {@code 0} if move was impossible.
+     */
+    public long move(long size) {
+        return buf.move(size);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(int b) {
+        writeByte(b);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(byte[] b) {
+        write(b, 0, b.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(byte[] b, int off, int len) {
+        UNSAFE.copyMemory(b, BYTE_ARR_OFF + off, null, move(len), len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(boolean v) {
+        writeByte(v ? 1 : 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(int v) {
+        mem.writeByte(move(1), (byte)v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(int v) {
+        mem.writeShort(move(2), (short)v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(int v) {
+        writeShort(v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(int v) {
+        mem.writeInt(move(4), v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(long v) {
+        mem.writeLong(move(8), v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(float v) {
+        mem.writeFloat(move(4), v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(double v) {
+        mem.writeDouble(move(8), v);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBytes(String s) {
+        writeUTF(s);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChars(String s) {
+        writeUTF(s);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUTF(String s) {
+        byte[] b = s.getBytes(StandardCharsets.UTF_8);
+
+        writeInt(b.length);
+        write(b);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java
new file mode 100644
index 0000000..a8e7a33
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/HadoopOffheapBuffer.java
@@ -0,0 +1,122 @@
+/*
+ * 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.shuffle.streams;
+
+/**
+ * Offheap buffer.
+ */
+public class HadoopOffheapBuffer {
+    /** Buffer begin address. */
+    private long bufPtr;
+
+    /** The first address we do not own. */
+    private long bufEnd;
+
+    /** Current read or write pointer. */
+    private long posPtr;
+
+    /**
+     * @param bufPtr Pointer to buffer begin.
+     * @param bufSize Size of the buffer.
+     */
+    public HadoopOffheapBuffer(long bufPtr, long bufSize) {
+        set(bufPtr, bufSize);
+    }
+
+    /**
+     * @param bufPtr Pointer to buffer begin.
+     * @param bufSize Size of the buffer.
+     */
+    public void set(long bufPtr, long bufSize) {
+        this.bufPtr = bufPtr;
+
+        posPtr = bufPtr;
+        bufEnd = bufPtr + bufSize;
+    }
+
+    /**
+     * @return Pointer to internal buffer begin.
+     */
+    public long begin() {
+        return bufPtr;
+    }
+
+    /**
+     * @return Buffer capacity.
+     */
+    public long capacity() {
+        return bufEnd - bufPtr;
+    }
+
+    /**
+     * @return Remaining capacity.
+     */
+    public long remaining() {
+        return bufEnd - posPtr;
+    }
+
+    /**
+     * @return Absolute pointer to the current position inside of the buffer.
+     */
+    public long pointer() {
+        return posPtr;
+    }
+
+    /**
+     * @param ptr Absolute pointer to the current position inside of the buffer.
+     */
+    public void pointer(long ptr) {
+        assert ptr >= bufPtr : bufPtr + " <= " + ptr;
+        assert ptr <= bufEnd : bufEnd + " <= " + bufPtr;
+
+        posPtr = ptr;
+    }
+
+    /**
+     * @param size Size move on.
+     * @return Old position pointer or {@code 0} if move goes beyond the end of the buffer.
+     */
+    public long move(long size) {
+        assert size > 0 : size;
+
+        long oldPos = posPtr;
+        long newPos = oldPos + size;
+
+        if (newPos > bufEnd)
+            return 0;
+
+        posPtr = newPos;
+
+        return oldPos;
+    }
+
+    /**
+     * @param ptr Pointer.
+     * @return {@code true} If the given pointer is inside of this buffer.
+     */
+    public boolean isInside(long ptr) {
+        return ptr >= bufPtr && ptr <= bufEnd;
+    }
+
+    /**
+     * Resets position to the beginning of buffer.
+     */
+    public void reset() {
+        posPtr = bufPtr;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorService.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorService.java
deleted file mode 100644
index 9ec637b..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorService.java
+++ /dev/null
@@ -1,232 +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.taskexecutor;
-
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.worker.*;
-import org.apache.ignite.thread.*;
-import org.jdk8.backport.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static java.util.Collections.*;
-
-/**
- * Executor service without thread pooling.
- */
-public class GridHadoopExecutorService {
-    /** */
-    private final LinkedBlockingQueue<Callable<?>> queue;
-
-    /** */
-    private final Collection<GridWorker> workers = newSetFromMap(new ConcurrentHashMap8<GridWorker, Boolean>());
-
-    /** */
-    private final AtomicInteger active = new AtomicInteger();
-
-    /** */
-    private final int maxTasks;
-
-    /** */
-    private final String gridName;
-
-    /** */
-    private final IgniteLogger log;
-
-    /** */
-    private volatile boolean shutdown;
-
-    /** */
-    private final GridWorkerListener lsnr = new GridWorkerListenerAdapter() {
-            @Override public void onStopped(GridWorker w) {
-                workers.remove(w);
-
-                if (shutdown) {
-                    active.decrementAndGet();
-
-                    return;
-                }
-
-                Callable<?> task = queue.poll();
-
-                if (task != null)
-                    startThread(task);
-                else {
-                    active.decrementAndGet();
-
-                    if (!queue.isEmpty())
-                        startFromQueue();
-                }
-            }
-        };
-
-    /**
-     * @param log Logger.
-     * @param gridName Grid name.
-     * @param maxTasks Max number of tasks.
-     * @param maxQueue Max queue length.
-     */
-    public GridHadoopExecutorService(IgniteLogger log, String gridName, int maxTasks, int maxQueue) {
-        assert maxTasks > 0 : maxTasks;
-        assert maxQueue > 0 : maxQueue;
-
-        this.maxTasks = maxTasks;
-        this.queue = new LinkedBlockingQueue<>(maxQueue);
-        this.gridName = gridName;
-        this.log = log.getLogger(GridHadoopExecutorService.class);
-    }
-
-    /**
-     * @return Number of active workers.
-     */
-    public int active() {
-        return workers.size();
-    }
-
-    /**
-     * Submit task.
-     *
-     * @param task Task.
-     */
-    public void submit(Callable<?> task) {
-        while (queue.isEmpty()) {
-            int active0 = active.get();
-
-            if (active0 == maxTasks)
-                break;
-
-            if (active.compareAndSet(active0, active0 + 1)) {
-                startThread(task);
-
-                return; // Started in new thread bypassing queue.
-            }
-        }
-
-        try {
-            while (!queue.offer(task, 100, TimeUnit.MILLISECONDS)) {
-                if (shutdown)
-                    return; // Rejected due to shutdown.
-            }
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            return;
-        }
-
-        startFromQueue();
-    }
-
-    /**
-     * Attempts to start task from queue.
-     */
-    private void startFromQueue() {
-        do {
-            int active0 = active.get();
-
-            if (active0 == maxTasks)
-                break;
-
-            if (active.compareAndSet(active0, active0 + 1)) {
-                Callable<?> task = queue.poll();
-
-                if (task == null) {
-                    int res = active.decrementAndGet();
-
-                    assert res >= 0 : res;
-
-                    break;
-                }
-
-                startThread(task);
-            }
-        }
-        while (!queue.isEmpty());
-    }
-
-    /**
-     * @param task Task.
-     */
-    private void startThread(final Callable<?> task) {
-        String workerName;
-
-        if (task instanceof GridHadoopRunnableTask) {
-            final GridHadoopTaskInfo i = ((GridHadoopRunnableTask)task).taskInfo();
-
-            workerName = "Hadoop-task-" + i.jobId() + "-" + i.type() + "-" + i.taskNumber() + "-" + i.attempt();
-        }
-        else
-            workerName = task.toString();
-
-        GridWorker w = new GridWorker(gridName, workerName, log, lsnr) {
-            @Override protected void body() {
-                try {
-                    task.call();
-                }
-                catch (Exception e) {
-                    log.error("Failed to execute task: " + task, e);
-                }
-            }
-        };
-
-        workers.add(w);
-
-        if (shutdown)
-            w.cancel();
-
-        new IgniteThread(w).start();
-    }
-
-    /**
-     * Shuts down this executor service.
-     *
-     * @param awaitTimeMillis Time in milliseconds to wait for tasks completion.
-     * @return {@code true} If all tasks completed.
-     */
-    public boolean shutdown(long awaitTimeMillis) {
-        shutdown = true;
-
-        for (GridWorker w : workers)
-            w.cancel();
-
-        while (awaitTimeMillis > 0 && !workers.isEmpty()) {
-            try {
-                Thread.sleep(100);
-
-                awaitTimeMillis -= 100;
-            }
-            catch (InterruptedException e) {
-                break;
-            }
-        }
-
-        return workers.isEmpty();
-    }
-
-    /**
-     * @return {@code true} If method {@linkplain #shutdown(long)} was already called.
-     */
-    public boolean isShutdown() {
-        return shutdown;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
index 0d49be9..1ce7d4a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
@@ -56,7 +56,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
     private long execEndTs;
 
     /** */
-    private GridHadoopMultimap combinerInput;
+    private HadoopMultimap combinerInput;
 
     /** */
     private volatile GridHadoopTaskContext ctx;
@@ -103,14 +103,14 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
 
         Throwable err = null;
 
-        GridHadoopTaskState state = GridHadoopTaskState.COMPLETED;
+        HadoopTaskState state = HadoopTaskState.COMPLETED;
 
-        GridHadoopPerformanceCounter perfCntr = null;
+        HadoopPerformanceCounter perfCntr = null;
 
         try {
             ctx = job.getTaskContext(info);
 
-            perfCntr = GridHadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
+            perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
 
             perfCntr.onTaskSubmit(info, submitTs);
             perfCntr.onTaskPrepare(info, execStartTs);
@@ -131,10 +131,10 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
             }
         }
         catch (HadoopTaskCancelledException ignored) {
-            state = GridHadoopTaskState.CANCELED;
+            state = HadoopTaskState.CANCELED;
         }
         catch (Throwable e) {
-            state = GridHadoopTaskState.FAILED;
+            state = HadoopTaskState.FAILED;
             err = e;
 
             U.error(log, "Task execution failed.", e);
@@ -145,7 +145,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
             if (perfCntr != null)
                 perfCntr.onTaskFinish(info, execEndTs);
 
-            onTaskFinished(new GridHadoopTaskStatus(state, err, ctx==null ? null : ctx.counters()));
+            onTaskFinished(new HadoopTaskStatus(state, err, ctx==null ? null : ctx.counters()));
 
             if (combinerInput != null)
                 combinerInput.close();
@@ -161,7 +161,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
      * @param perfCntr Performance counter.
      * @throws IgniteCheckedException If failed.
      */
-    private void runTask(GridHadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
+    private void runTask(HadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
         if (cancelled)
             throw new HadoopTaskCancelledException("Task cancelled.");
 
@@ -190,7 +190,7 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
     /**
      * @param status Task status.
      */
-    protected abstract void onTaskFinished(GridHadoopTaskStatus status);
+    protected abstract void onTaskFinished(HadoopTaskStatus status);
 
     /**
      * @param ctx Task context.
@@ -248,8 +248,8 @@ public abstract class GridHadoopRunnableTask implements Callable<Void> {
                     assert combinerInput == null;
 
                     combinerInput = get(job.info(), SHUFFLE_COMBINER_NO_SORTING, false) ?
-                        new GridHadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)):
-                        new GridHadoopSkipList(job.info(), mem); // TODO replace with red-black tree
+                        new HadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)):
+                        new HadoopSkipList(job.info(), mem); // TODO replace with red-black tree
 
                     return combinerInput.startAdding(ctx);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskState.java
deleted file mode 100644
index d1eaa66..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskState.java
+++ /dev/null
@@ -1,38 +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.taskexecutor;
-
-/**
-* State of the task.
-*/
-public enum GridHadoopTaskState {
-    /** Running task. */
-    RUNNING,
-
-    /** Completed task. */
-    COMPLETED,
-
-    /** Failed task. */
-    FAILED,
-
-    /** Canceled task. */
-    CANCELED,
-
-    /** Process crashed. */
-    CRASHED
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskStatus.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskStatus.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskStatus.java
deleted file mode 100644
index 89ef8c1..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopTaskStatus.java
+++ /dev/null
@@ -1,114 +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.taskexecutor;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Task status.
- */
-public class GridHadoopTaskStatus implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private GridHadoopTaskState state;
-
-    /** */
-    private Throwable failCause;
-
-    /** */
-    private GridHadoopCounters cntrs;
-
-    /**
-     * Default constructor required by {@link Externalizable}.
-     */
-    public GridHadoopTaskStatus() {
-        // No-op.
-    }
-
-    /**
-     * Creates new instance.
-     *
-     * @param state Task state.
-     * @param failCause Failure cause (if any).
-     */
-    public GridHadoopTaskStatus(GridHadoopTaskState state, @Nullable Throwable failCause) {
-        this(state, failCause, null);
-    }
-
-    /**
-     * Creates new instance.
-     *
-     * @param state Task state.
-     * @param failCause Failure cause (if any).
-     * @param cntrs Task counters.
-     */
-    public GridHadoopTaskStatus(GridHadoopTaskState state, @Nullable Throwable failCause,
-        @Nullable GridHadoopCounters cntrs) {
-        assert state != null;
-
-        this.state = state;
-        this.failCause = failCause;
-        this.cntrs = cntrs;
-    }
-
-    /**
-     * @return State.
-     */
-    public GridHadoopTaskState state() {
-        return state;
-    }
-
-    /**
-     * @return Fail cause.
-     */
-    @Nullable public Throwable failCause() {
-        return failCause;
-    }
-
-    /**
-     * @return Counters.
-     */
-    @Nullable public GridHadoopCounters counters() {
-        return cntrs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopTaskStatus.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeObject(state);
-        out.writeObject(failCause);
-        out.writeObject(cntrs);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        state = (GridHadoopTaskState)in.readObject();
-        failCause = (Throwable)in.readObject();
-        cntrs = (GridHadoopCounters)in.readObject();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
index f896daa..934ff35 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
@@ -38,7 +38,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
     private final ConcurrentMap<GridHadoopJobId, Collection<GridHadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
 
     /** Executor service to run tasks. */
-    private GridHadoopExecutorService exec;
+    private HadoopExecutorService exec;
 
     /** {@inheritDoc} */
     @Override public void onKernalStart() throws IgniteCheckedException {
@@ -46,7 +46,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
 
         jobTracker = ctx.jobTracker();
 
-        exec = new GridHadoopExecutorService(log, ctx.kernalContext().gridName(),
+        exec = new HadoopExecutorService(log, ctx.kernalContext().gridName(),
             ctx.configuration().getMaxParallelTasks(), ctx.configuration().getMaxTaskQueueSize());
     }
 
@@ -91,7 +91,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
 
             GridHadoopRunnableTask task = new GridHadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
                 ctx.localNodeId()) {
-                @Override protected void onTaskFinished(GridHadoopTaskStatus status) {
+                @Override protected void onTaskFinished(HadoopTaskStatus status) {
                     if (log.isDebugEnabled())
                         log.debug("Finished task execution [jobId=" + job.id() + ", taskInfo=" + info + ", " +
                             "waitTime=" + waitTime() + ", execTime=" + executionTime() + ']');
@@ -136,7 +136,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException {
+    @Override public void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException {
         if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
             Collection<GridHadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
new file mode 100644
index 0000000..19f903f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
@@ -0,0 +1,231 @@
+/*
+ * 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.taskexecutor;
+
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.thread.*;
+import org.jdk8.backport.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.Collections.*;
+
+/**
+ * Executor service without thread pooling.
+ */
+public class HadoopExecutorService {
+    /** */
+    private final LinkedBlockingQueue<Callable<?>> queue;
+
+    /** */
+    private final Collection<GridWorker> workers = newSetFromMap(new ConcurrentHashMap8<GridWorker, Boolean>());
+
+    /** */
+    private final AtomicInteger active = new AtomicInteger();
+
+    /** */
+    private final int maxTasks;
+
+    /** */
+    private final String gridName;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private volatile boolean shutdown;
+
+    /** */
+    private final GridWorkerListener lsnr = new GridWorkerListenerAdapter() {
+            @Override public void onStopped(GridWorker w) {
+                workers.remove(w);
+
+                if (shutdown) {
+                    active.decrementAndGet();
+
+                    return;
+                }
+
+                Callable<?> task = queue.poll();
+
+                if (task != null)
+                    startThread(task);
+                else {
+                    active.decrementAndGet();
+
+                    if (!queue.isEmpty())
+                        startFromQueue();
+                }
+            }
+        };
+
+    /**
+     * @param log Logger.
+     * @param gridName Grid name.
+     * @param maxTasks Max number of tasks.
+     * @param maxQueue Max queue length.
+     */
+    public HadoopExecutorService(IgniteLogger log, String gridName, int maxTasks, int maxQueue) {
+        assert maxTasks > 0 : maxTasks;
+        assert maxQueue > 0 : maxQueue;
+
+        this.maxTasks = maxTasks;
+        this.queue = new LinkedBlockingQueue<>(maxQueue);
+        this.gridName = gridName;
+        this.log = log.getLogger(HadoopExecutorService.class);
+    }
+
+    /**
+     * @return Number of active workers.
+     */
+    public int active() {
+        return workers.size();
+    }
+
+    /**
+     * Submit task.
+     *
+     * @param task Task.
+     */
+    public void submit(Callable<?> task) {
+        while (queue.isEmpty()) {
+            int active0 = active.get();
+
+            if (active0 == maxTasks)
+                break;
+
+            if (active.compareAndSet(active0, active0 + 1)) {
+                startThread(task);
+
+                return; // Started in new thread bypassing queue.
+            }
+        }
+
+        try {
+            while (!queue.offer(task, 100, TimeUnit.MILLISECONDS)) {
+                if (shutdown)
+                    return; // Rejected due to shutdown.
+            }
+        }
+        catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+
+            return;
+        }
+
+        startFromQueue();
+    }
+
+    /**
+     * Attempts to start task from queue.
+     */
+    private void startFromQueue() {
+        do {
+            int active0 = active.get();
+
+            if (active0 == maxTasks)
+                break;
+
+            if (active.compareAndSet(active0, active0 + 1)) {
+                Callable<?> task = queue.poll();
+
+                if (task == null) {
+                    int res = active.decrementAndGet();
+
+                    assert res >= 0 : res;
+
+                    break;
+                }
+
+                startThread(task);
+            }
+        }
+        while (!queue.isEmpty());
+    }
+
+    /**
+     * @param task Task.
+     */
+    private void startThread(final Callable<?> task) {
+        String workerName;
+
+        if (task instanceof GridHadoopRunnableTask) {
+            final GridHadoopTaskInfo i = ((GridHadoopRunnableTask)task).taskInfo();
+
+            workerName = "Hadoop-task-" + i.jobId() + "-" + i.type() + "-" + i.taskNumber() + "-" + i.attempt();
+        }
+        else
+            workerName = task.toString();
+
+        GridWorker w = new GridWorker(gridName, workerName, log, lsnr) {
+            @Override protected void body() {
+                try {
+                    task.call();
+                }
+                catch (Exception e) {
+                    log.error("Failed to execute task: " + task, e);
+                }
+            }
+        };
+
+        workers.add(w);
+
+        if (shutdown)
+            w.cancel();
+
+        new IgniteThread(w).start();
+    }
+
+    /**
+     * Shuts down this executor service.
+     *
+     * @param awaitTimeMillis Time in milliseconds to wait for tasks completion.
+     * @return {@code true} If all tasks completed.
+     */
+    public boolean shutdown(long awaitTimeMillis) {
+        shutdown = true;
+
+        for (GridWorker w : workers)
+            w.cancel();
+
+        while (awaitTimeMillis > 0 && !workers.isEmpty()) {
+            try {
+                Thread.sleep(100);
+
+                awaitTimeMillis -= 100;
+            }
+            catch (InterruptedException e) {
+                break;
+            }
+        }
+
+        return workers.isEmpty();
+    }
+
+    /**
+     * @return {@code true} If method {@linkplain #shutdown(long)} was already called.
+     */
+    public boolean isShutdown() {
+        return shutdown;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
index a3d3bf7..2da2373 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
@@ -53,5 +53,5 @@ public abstract class HadoopTaskExecutorAdapter extends HadoopComponent {
      *
      * @param meta Job metadata.
      */
-    public abstract void onJobStateChanged(GridHadoopJobMetadata meta) throws IgniteCheckedException;
+    public abstract void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskState.java
new file mode 100644
index 0000000..cf2a28e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskState.java
@@ -0,0 +1,38 @@
+/*
+ * 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.taskexecutor;
+
+/**
+* State of the task.
+*/
+public enum HadoopTaskState {
+    /** Running task. */
+    RUNNING,
+
+    /** Completed task. */
+    COMPLETED,
+
+    /** Failed task. */
+    FAILED,
+
+    /** Canceled task. */
+    CANCELED,
+
+    /** Process crashed. */
+    CRASHED
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskStatus.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskStatus.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskStatus.java
new file mode 100644
index 0000000..490f0b2
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskStatus.java
@@ -0,0 +1,114 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Task status.
+ */
+public class HadoopTaskStatus implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private HadoopTaskState state;
+
+    /** */
+    private Throwable failCause;
+
+    /** */
+    private GridHadoopCounters cntrs;
+
+    /**
+     * Default constructor required by {@link Externalizable}.
+     */
+    public HadoopTaskStatus() {
+        // No-op.
+    }
+
+    /**
+     * Creates new instance.
+     *
+     * @param state Task state.
+     * @param failCause Failure cause (if any).
+     */
+    public HadoopTaskStatus(HadoopTaskState state, @Nullable Throwable failCause) {
+        this(state, failCause, null);
+    }
+
+    /**
+     * Creates new instance.
+     *
+     * @param state Task state.
+     * @param failCause Failure cause (if any).
+     * @param cntrs Task counters.
+     */
+    public HadoopTaskStatus(HadoopTaskState state, @Nullable Throwable failCause,
+        @Nullable GridHadoopCounters cntrs) {
+        assert state != null;
+
+        this.state = state;
+        this.failCause = failCause;
+        this.cntrs = cntrs;
+    }
+
+    /**
+     * @return State.
+     */
+    public HadoopTaskState state() {
+        return state;
+    }
+
+    /**
+     * @return Fail cause.
+     */
+    @Nullable public Throwable failCause() {
+        return failCause;
+    }
+
+    /**
+     * @return Counters.
+     */
+    @Nullable public GridHadoopCounters counters() {
+        return cntrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopTaskStatus.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(state);
+        out.writeObject(failCause);
+        out.writeObject(cntrs);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        state = (HadoopTaskState)in.readObject();
+        failCause = (Throwable)in.readObject();
+        cntrs = (GridHadoopCounters)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskMetadata.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskMetadata.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskMetadata.java
deleted file mode 100644
index 39606bc..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskMetadata.java
+++ /dev/null
@@ -1,68 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-
-/**
- * External task metadata (classpath, JVM options) needed to start external process execution.
- */
-public class GridHadoopExternalTaskMetadata {
-    /** Process classpath. */
-    private Collection<String> classpath;
-
-    /** JVM options. */
-    @GridToStringInclude
-    private Collection<String> jvmOpts;
-
-    /**
-     * @return JVM Options.
-     */
-    public Collection<String> jvmOptions() {
-        return jvmOpts;
-    }
-
-    /**
-     * @param jvmOpts JVM options.
-     */
-    public void jvmOptions(Collection<String> jvmOpts) {
-        this.jvmOpts = jvmOpts;
-    }
-
-    /**
-     * @return Classpath.
-     */
-    public Collection<String> classpath() {
-        return classpath;
-    }
-
-    /**
-     * @param classpath Classpath.
-     */
-    public void classpath(Collection<String> classpath) {
-        this.classpath = classpath;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopExternalTaskMetadata.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopJobInfoUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopJobInfoUpdateRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopJobInfoUpdateRequest.java
deleted file mode 100644
index 2a7c7a8..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopJobInfoUpdateRequest.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.hadoop.taskexecutor.external;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Job info update request.
- */
-public class GridHadoopJobInfoUpdateRequest implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Job ID. */
-    @GridToStringInclude
-    private GridHadoopJobId jobId;
-
-    /** Job phase. */
-    @GridToStringInclude
-    private GridHadoopJobPhase jobPhase;
-
-    /** Reducers addresses. */
-    @GridToStringInclude
-    private GridHadoopProcessDescriptor[] reducersAddrs;
-
-    /**
-     * Constructor required by {@link Externalizable}.
-     */
-    public GridHadoopJobInfoUpdateRequest() {
-        // No-op.
-    }
-
-    /**
-     * @param jobId Job ID.
-     * @param jobPhase Job phase.
-     * @param reducersAddrs Reducers addresses.
-     */
-    public GridHadoopJobInfoUpdateRequest(GridHadoopJobId jobId, GridHadoopJobPhase jobPhase,
-        GridHadoopProcessDescriptor[] reducersAddrs) {
-        assert jobId != null;
-
-        this.jobId = jobId;
-        this.jobPhase = jobPhase;
-        this.reducersAddrs = reducersAddrs;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @return Job phase.
-     */
-    public GridHadoopJobPhase jobPhase() {
-        return jobPhase;
-    }
-
-    /**
-     * @return Reducers addresses.
-     */
-    public GridHadoopProcessDescriptor[] reducersAddresses() {
-        return reducersAddrs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        jobId.writeExternal(out);
-
-        out.writeObject(jobPhase);
-        U.writeArray(out, reducersAddrs);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = new GridHadoopJobId();
-        jobId.readExternal(in);
-
-        jobPhase = (GridHadoopJobPhase)in.readObject();
-        reducersAddrs = (GridHadoopProcessDescriptor[])U.readArray(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopJobInfoUpdateRequest.class, this);
-    }
-}


[31/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (5).

Posted by vo...@apache.org.
# IGNITE-386: WIP on internal namings (5).


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

Branch: refs/heads/ignite-386
Commit: e1ffc10f6332acb3d8b8b29f6602cfdc0d45be3c
Parents: 17c8d0d
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 3 16:08:03 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 3 16:08:03 2015 +0300

----------------------------------------------------------------------
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |   6 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   4 +-
 .../igfs/hadoop/HadoopIgfsInputStream.java      | 626 +++++++++++++++++++
 .../internal/igfs/hadoop/HadoopIgfsIpcIo.java   | 599 ++++++++++++++++++
 .../igfs/hadoop/HadoopIgfsIpcIoListener.java    |   2 +-
 .../internal/igfs/hadoop/HadoopIgfsOutProc.java | 466 ++++++++++++++
 .../igfs/hadoop/HadoopIgfsOutputStream.java     | 201 ++++++
 .../internal/igfs/hadoop/HadoopIgfsWrapper.java |   6 +-
 .../igfs/hadoop/HadoopInputIgfsStream.java      | 626 -------------------
 .../internal/igfs/hadoop/HadoopIpcIgfsIo.java   | 599 ------------------
 .../internal/igfs/hadoop/HadoopOutProcIgfs.java | 466 --------------
 .../igfs/hadoop/HadoopOutputIgfsStream.java     | 201 ------
 .../ignite/internal/igfs/hadoop/package.html    |  24 -
 .../apache/ignite/internal/igfs/package.html    |  24 -
 14 files changed, 1901 insertions(+), 1949 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 9c95437..f7b5dda 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -528,7 +528,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                     LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
                         ", bufSize=" + bufSize + ']');
 
-                HadoopInputIgfsStream igfsIn = new HadoopInputIgfsStream(stream, stream.length(),
+                HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
                     bufSize, LOG, clientLog, logId);
 
                 if (LOG.isDebugEnabled())
@@ -599,7 +599,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                 if (LOG.isDebugEnabled())
                     LOG.debug("Opened output stream in create [path=" + path + ", delegate=" + stream + ']');
 
-                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG, clientLog,
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
                     logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);
@@ -673,7 +673,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
                 if (LOG.isDebugEnabled())
                     LOG.debug("Opened output stream in append [path=" + path + ", delegate=" + stream + ']');
 
-                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG, clientLog,
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG, clientLog,
                     logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index 1c9165c..016a068 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -467,7 +467,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
                     LOG.debug("Opening input stream [thread=" + Thread.currentThread().getName() + ", path=" + path +
                         ", bufSize=" + bufSize + ']');
 
-                HadoopInputIgfsStream igfsIn = new HadoopInputIgfsStream(stream, stream.length(),
+                HadoopIgfsInputStream igfsIn = new HadoopIgfsInputStream(stream, stream.length(),
                     bufSize, LOG, clientLog, logId);
 
                 if (LOG.isDebugEnabled())
@@ -566,7 +566,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
                 assert stream != null;
 
-                HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(stream, LOG,
+                HadoopIgfsOutputStream igfsOut = new HadoopIgfsOutputStream(stream, LOG,
                     clientLog, logId);
 
                 bufSize = Math.max(64 * 1024, bufSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInputStream.java
new file mode 100644
index 0000000..6293e2f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsInputStream.java
@@ -0,0 +1,626 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * IGFS input stream wrapper for hadoop interfaces.
+ */
+@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+public final class HadoopIgfsInputStream extends InputStream implements Seekable, PositionedReadable,
+    HadoopIgfsStreamEventListener {
+    /** Minimum buffer size. */
+    private static final int MIN_BUF_SIZE = 4 * 1024;
+
+    /** Server stream delegate. */
+    private HadoopIgfsStreamDelegate delegate;
+
+    /** Stream ID used by logger. */
+    private long logStreamId;
+
+    /** Stream position. */
+    private long pos;
+
+    /** Stream read limit. */
+    private long limit;
+
+    /** Mark position. */
+    private long markPos = -1;
+
+    /** Prefetch buffer. */
+    private DoubleFetchBuffer buf = new DoubleFetchBuffer();
+
+    /** Buffer half size for double-buffering. */
+    private int bufHalfSize;
+
+    /** Closed flag. */
+    private volatile boolean closed;
+
+    /** Flag set if stream was closed due to connection breakage. */
+    private boolean connBroken;
+
+    /** Logger. */
+    private Log log;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Read time. */
+    private long readTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of read bytes. */
+    private long total;
+
+    /**
+     * Creates input stream.
+     *
+     * @param delegate Server stream delegate.
+     * @param limit Read limit.
+     * @param bufSize Buffer size.
+     * @param log Log.
+     * @param clientLog Client logger.
+     */
+    public HadoopIgfsInputStream(HadoopIgfsStreamDelegate delegate, long limit, int bufSize, Log log,
+        IgfsLogger clientLog, long logStreamId) {
+        assert limit >= 0;
+
+        this.delegate = delegate;
+        this.limit = limit;
+        this.log = log;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        bufHalfSize = Math.max(bufSize, MIN_BUF_SIZE);
+
+        lastTs = System.nanoTime();
+
+        delegate.hadoop().addEventListener(delegate, this);
+    }
+
+    /**
+     * Read start.
+     */
+    private void readStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void readEnd() {
+        long now = System.nanoTime();
+
+        readTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read() throws IOException {
+        checkClosed();
+
+        readStart();
+
+        try {
+            if (eof())
+                return -1;
+
+            buf.refreshAhead(pos);
+
+            int res = buf.atPosition(pos);
+
+            pos++;
+            total++;
+
+            buf.refreshAhead(pos);
+
+            return res;
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(@NotNull byte[] b, int off, int len) throws IOException {
+        checkClosed();
+
+        if (eof())
+            return -1;
+
+        readStart();
+
+        try {
+            long remaining = limit - pos;
+
+            int read = buf.flatten(b, pos, off, len);
+
+            pos += read;
+            total += read;
+            remaining -= read;
+
+            if (remaining > 0 && read != len) {
+                int readAmt = (int)Math.min(remaining, len - read);
+
+                delegate.hadoop().readData(delegate, pos, readAmt, b, off + read, len - read).get();
+
+                read += readAmt;
+                pos += readAmt;
+                total += readAmt;
+            }
+
+            buf.refreshAhead(pos);
+
+            return read;
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long skip(long n) throws IOException {
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSkip(logStreamId, n);
+
+        long oldPos = pos;
+
+        if (pos + n <= limit)
+            pos += n;
+        else
+            pos = limit;
+
+        buf.refreshAhead(pos);
+
+        return pos - oldPos;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int available() throws IOException {
+        checkClosed();
+
+        int available = buf.available(pos);
+
+        assert available >= 0;
+
+        return available;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void close() throws IOException {
+        if (!closed) {
+            readStart();
+
+            if (log.isDebugEnabled())
+                log.debug("Closing input stream: " + delegate);
+
+            delegate.hadoop().closeStream(delegate);
+
+            readEnd();
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
+
+            markClosed(false);
+
+            if (log.isDebugEnabled())
+                log.debug("Closed stream [delegate=" + delegate + ", readTime=" + readTime +
+                    ", userTime=" + userTime + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void mark(int readLimit) {
+        markPos = pos;
+
+        if (clientLog.isLogEnabled())
+            clientLog.logMark(logStreamId, readLimit);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void reset() throws IOException {
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logReset(logStreamId);
+
+        if (markPos == -1)
+            throw new IOException("Stream was not marked.");
+
+        pos = markPos;
+
+        buf.refreshAhead(pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean markSupported() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(long position, byte[] buf, int off, int len) throws IOException {
+        long remaining = limit - position;
+
+        int read = (int)Math.min(len, remaining);
+
+        // Return -1 at EOF.
+        if (read == 0)
+            return -1;
+
+        readFully(position, buf, off, read);
+
+        return read;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void readFully(long position, byte[] buf, int off, int len) throws IOException {
+        long remaining = limit - position;
+
+        checkClosed();
+
+        if (len > remaining)
+            throw new EOFException("End of stream reached before data was fully read.");
+
+        readStart();
+
+        try {
+            int read = this.buf.flatten(buf, position, off, len);
+
+            total += read;
+
+            if (read != len) {
+                int readAmt = len - read;
+
+                delegate.hadoop().readData(delegate, position + read, readAmt, buf, off + read, readAmt).get();
+
+                total += readAmt;
+            }
+
+            if (clientLog.isLogEnabled())
+                clientLog.logRandomRead(logStreamId, position, len);
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(long position, byte[] buf) throws IOException {
+        readFully(position, buf, 0, buf.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void seek(long pos) throws IOException {
+        A.ensure(pos >= 0, "position must be non-negative");
+
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSeek(logStreamId, pos);
+
+        if (pos > limit)
+            pos = limit;
+
+        if (log.isDebugEnabled())
+            log.debug("Seek to position [delegate=" + delegate + ", pos=" + pos + ", oldPos=" + this.pos + ']');
+
+        this.pos = pos;
+
+        buf.refreshAhead(pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long getPos() {
+        return pos;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean seekToNewSource(long targetPos) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        markClosed(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(String errMsg) {
+        // No-op.
+    }
+
+    /**
+     * Marks stream as closed.
+     *
+     * @param connBroken {@code True} if connection with server was lost.
+     */
+    private void markClosed(boolean connBroken) {
+        // It is ok to have race here.
+        if (!closed) {
+            closed = true;
+
+            this.connBroken = connBroken;
+
+            delegate.hadoop().removeEventListener(delegate);
+        }
+    }
+
+    /**
+     * @throws IOException If check failed.
+     */
+    private void checkClosed() throws IOException {
+        if (closed) {
+            if (connBroken)
+                throw new IOException("Server connection was lost.");
+            else
+                throw new IOException("Stream is closed.");
+        }
+    }
+
+    /**
+     * @return {@code True} if end of stream reached.
+     */
+    private boolean eof() {
+        return limit == pos;
+    }
+
+    /**
+     * Asynchronous prefetch buffer.
+     */
+    private static class FetchBufferPart {
+        /** Read future. */
+        private GridPlainFuture<byte[]> readFut;
+
+        /** Position of cached chunk in file. */
+        private long pos;
+
+        /** Prefetch length. Need to store as read future result might be not available yet. */
+        private int len;
+
+        /**
+         * Creates fetch buffer part.
+         *
+         * @param readFut Read future for this buffer.
+         * @param pos Read position.
+         * @param len Chunk length.
+         */
+        private FetchBufferPart(GridPlainFuture<byte[]> readFut, long pos, int len) {
+            this.readFut = readFut;
+            this.pos = pos;
+            this.len = len;
+        }
+
+        /**
+         * Copies cached data if specified position matches cached region.
+         *
+         * @param dst Destination buffer.
+         * @param pos Read position in file.
+         * @param dstOff Offset in destination buffer from which start writing.
+         * @param len Maximum number of bytes to copy.
+         * @return Number of bytes copied.
+         * @throws IgniteCheckedException If read future failed.
+         */
+        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
+            // If read start position is within cached boundaries.
+            if (contains(pos)) {
+                byte[] data = readFut.get();
+
+                int srcPos = (int)(pos - this.pos);
+                int cpLen = Math.min(len, data.length - srcPos);
+
+                U.arrayCopy(data, srcPos, dst, dstOff, cpLen);
+
+                return cpLen;
+            }
+
+            return 0;
+        }
+
+        /**
+         * @return {@code True} if data is ready to be read.
+         */
+        public boolean ready() {
+            return readFut.isDone();
+        }
+
+        /**
+         * Checks if current buffer part contains given position.
+         *
+         * @param pos Position to check.
+         * @return {@code True} if position matches buffer region.
+         */
+        public boolean contains(long pos) {
+            return this.pos <= pos && this.pos + len > pos;
+        }
+    }
+
+    private class DoubleFetchBuffer {
+        /**  */
+        private FetchBufferPart first;
+
+        /** */
+        private FetchBufferPart second;
+
+        /**
+         * Copies fetched data from both buffers to destination array if cached region matched read position.
+         *
+         * @param dst Destination buffer.
+         * @param pos Read position in file.
+         * @param dstOff Destination buffer offset.
+         * @param len Maximum number of bytes to copy.
+         * @return Number of bytes copied.
+         * @throws IgniteCheckedException If any read operation failed.
+         */
+        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
+            assert dstOff >= 0;
+            assert dstOff + len <= dst.length : "Invalid indices [dst.length=" + dst.length + ", dstOff=" + dstOff +
+                ", len=" + len + ']';
+
+            int bytesCopied = 0;
+
+            if (first != null) {
+                bytesCopied += first.flatten(dst, pos, dstOff, len);
+
+                if (bytesCopied != len && second != null) {
+                    assert second.pos == first.pos + first.len;
+
+                    bytesCopied += second.flatten(dst, pos + bytesCopied, dstOff + bytesCopied, len - bytesCopied);
+                }
+            }
+
+            return bytesCopied;
+        }
+
+        /**
+         * Gets byte at specified position in buffer.
+         *
+         * @param pos Stream position.
+         * @return Read byte.
+         * @throws IgniteCheckedException If read failed.
+         */
+        public int atPosition(long pos) throws IgniteCheckedException {
+            // Should not reach here if stream contains no data.
+            assert first != null;
+
+            if (first.contains(pos)) {
+                byte[] bytes = first.readFut.get();
+
+                return bytes[((int)(pos - first.pos))] & 0xFF;
+            }
+            else {
+                assert second != null;
+                assert second.contains(pos);
+
+                byte[] bytes = second.readFut.get();
+
+                return bytes[((int)(pos - second.pos))] & 0xFF;
+            }
+        }
+
+        /**
+         * Starts asynchronous buffer refresh if needed, depending on current position.
+         *
+         * @param pos Current stream position.
+         */
+        public void refreshAhead(long pos) {
+            if (fullPrefetch(pos)) {
+                first = fetch(pos, bufHalfSize);
+                second = fetch(pos + bufHalfSize, bufHalfSize);
+            }
+            else if (needFlip(pos)) {
+                first = second;
+
+                second = fetch(first.pos + first.len, bufHalfSize);
+            }
+        }
+
+        /**
+         * @param pos Position from which read is expected.
+         * @return Number of bytes available to be read without blocking.
+         */
+        public int available(long pos) {
+            int available = 0;
+
+            if (first != null) {
+                if (first.contains(pos)) {
+                    if (first.ready()) {
+                        available += (pos - first.pos);
+
+                        if (second != null && second.ready())
+                            available += second.len;
+                    }
+                }
+                else {
+                    if (second != null && second.contains(pos) && second.ready())
+                        available += (pos - second.pos);
+                }
+            }
+
+            return available;
+        }
+
+        /**
+         * Checks if position shifted enough to forget previous buffer.
+         *
+         * @param pos Current position.
+         * @return {@code True} if need flip buffers.
+         */
+        private boolean needFlip(long pos) {
+            // Return true if we read more then half of second buffer.
+            return second != null && second.contains(pos);
+        }
+
+        /**
+         * Determines if all cached bytes should be discarded and new region should be
+         * prefetched.
+         *
+         * @param curPos Current stream position.
+         * @return {@code True} if need to refresh both blocks.
+         */
+        private boolean fullPrefetch(long curPos) {
+            // If no data was prefetched yet, return true.
+            return first == null || curPos < first.pos || (second != null && curPos >= second.pos + second.len);
+        }
+
+        /**
+         * Starts asynchronous fetch for given region.
+         *
+         * @param pos Position to read from.
+         * @param size Number of bytes to read.
+         * @return Fetch buffer part.
+         */
+        private FetchBufferPart fetch(long pos, int size) {
+            long remaining = limit - pos;
+
+            size = (int)Math.min(size, remaining);
+
+            return size <= 0 ? null :
+                new FetchBufferPart(delegate.hadoop().readData(delegate, pos, size, null, 0, 0), pos, size);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIo.java
new file mode 100644
index 0000000..47e5763
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIo.java
@@ -0,0 +1,599 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * IO layer implementation based on blocking IPC streams.
+ */
+@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+public class HadoopIgfsIpcIo implements HadoopIgfsIo {
+    /** Logger. */
+    private Log log;
+
+    /** Request futures map. */
+    private ConcurrentMap<Long, HadoopIgfsFuture> reqMap =
+        new ConcurrentHashMap8<>();
+
+    /** Request ID counter. */
+    private AtomicLong reqIdCnt = new AtomicLong();
+
+    /** Endpoint. */
+    private IpcEndpoint endpoint;
+
+    /** Endpoint output stream. */
+    private IgfsDataOutputStream out;
+
+    /** Protocol. */
+    private final IgfsMarshaller marsh;
+
+    /** Client reader thread. */
+    private Thread reader;
+
+    /** Lock for graceful shutdown. */
+    private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
+
+    /** Stopping flag. */
+    private volatile boolean stopping;
+
+    /** Server endpoint address. */
+    private final String endpointAddr;
+
+    /** Number of open file system sessions. */
+    private final AtomicInteger activeCnt = new AtomicInteger(1);
+
+    /** Event listeners. */
+    private final Collection<HadoopIgfsIpcIoListener> lsnrs =
+        new GridConcurrentHashSet<>();
+
+    /** Cached connections. */
+    private static final ConcurrentMap<String, HadoopIgfsIpcIo> ipcCache =
+        new ConcurrentHashMap8<>();
+
+    /** Striped lock that prevents multiple instance creation in {@link #get(Log, String)}. */
+    private static final GridStripedLock initLock = new GridStripedLock(32);
+
+    /**
+     * @param endpointAddr Endpoint.
+     * @param marsh Protocol.
+     * @param log Logger to use.
+     */
+    public HadoopIgfsIpcIo(String endpointAddr, IgfsMarshaller marsh, Log log) {
+        assert endpointAddr != null;
+        assert marsh != null;
+
+        this.endpointAddr = endpointAddr;
+        this.marsh = marsh;
+        this.log = log;
+    }
+
+    /**
+     * Returns a started and valid instance of this class
+     * for a given endpoint.
+     *
+     * @param log Logger to use for new instance.
+     * @param endpoint Endpoint string.
+     * @return New or existing cached instance, which is started and operational.
+     * @throws IOException If new instance was created but failed to start.
+     */
+    public static HadoopIgfsIpcIo get(Log log, String endpoint) throws IOException {
+        while (true) {
+            HadoopIgfsIpcIo clientIo = ipcCache.get(endpoint);
+
+            if (clientIo != null) {
+                if (clientIo.acquire())
+                    return clientIo;
+                else
+                    // If concurrent close.
+                    ipcCache.remove(endpoint, clientIo);
+            }
+            else {
+                Lock lock = initLock.getLock(endpoint);
+
+                lock.lock();
+
+                try {
+                    clientIo = ipcCache.get(endpoint);
+
+                    if (clientIo != null) { // Perform double check.
+                        if (clientIo.acquire())
+                            return clientIo;
+                        else
+                            // If concurrent close.
+                            ipcCache.remove(endpoint, clientIo);
+                    }
+
+                    // Otherwise try creating a new one.
+                    clientIo = new HadoopIgfsIpcIo(endpoint, new IgfsMarshaller(), log);
+
+                    try {
+                        clientIo.start();
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IOException(e.getMessage(), e);
+                    }
+
+                    HadoopIgfsIpcIo old = ipcCache.putIfAbsent(endpoint, clientIo);
+
+                    // Put in exclusive lock.
+                    assert old == null;
+
+                    return clientIo;
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        }
+    }
+
+    /**
+     * Increases usage count for this instance.
+     *
+     * @return {@code true} if usage count is greater than zero.
+     */
+    private boolean acquire() {
+        while (true) {
+            int cnt = activeCnt.get();
+
+            if (cnt == 0) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO not acquired (count was 0): " + this);
+
+                return false;
+            }
+
+            // Need to make sure that no-one decremented count in between.
+            if (activeCnt.compareAndSet(cnt, cnt + 1)) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO acquired: " + this);
+
+                return true;
+            }
+        }
+    }
+
+    /**
+     * Releases this instance, decrementing usage count.
+     * <p>
+     * If usage count becomes zero, the instance is stopped
+     * and removed from cache.
+     */
+    public void release() {
+        while (true) {
+            int cnt = activeCnt.get();
+
+            if (cnt == 0) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO not released (count was 0): " + this);
+
+                return;
+            }
+
+            if (activeCnt.compareAndSet(cnt, cnt - 1)) {
+                if (cnt == 1) {
+                    ipcCache.remove(endpointAddr, this);
+
+                    if (log.isDebugEnabled())
+                        log.debug("IPC IO stopping as unused: " + this);
+
+                    stop();
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("IPC IO released: " + this);
+
+                return;
+            }
+        }
+    }
+
+    /**
+     * Closes this IO instance, removing it from cache.
+     */
+    public void forceClose() {
+        if (ipcCache.remove(endpointAddr, this))
+            stop();
+    }
+
+    /**
+     * Starts the IO.
+     *
+     * @throws IgniteCheckedException If failed to connect the endpoint.
+     */
+    private void start() throws IgniteCheckedException {
+        boolean success = false;
+
+        try {
+            endpoint = IpcEndpointFactory.connectEndpoint(
+                    endpointAddr, new GridLoggerProxy(new HadoopIgfsJclLogger(log), null, null, ""));
+
+            out = new IgfsDataOutputStream(new BufferedOutputStream(endpoint.outputStream()));
+
+            reader = new ReaderThread();
+
+            // Required for Hadoop 2.x
+            reader.setDaemon(true);
+
+            reader.start();
+
+            success = true;
+        }
+        catch (IgniteCheckedException e) {
+            IpcOutOfSystemResourcesException resEx = e.getCause(IpcOutOfSystemResourcesException.class);
+
+            if (resEx != null)
+                throw new IgniteCheckedException(IpcSharedMemoryServerEndpoint.OUT_OF_RESOURCES_MSG, resEx);
+
+            throw e;
+        }
+        finally {
+            if (!success)
+                stop();
+        }
+    }
+
+    /**
+     * Shuts down the IO. No send requests will be accepted anymore, all pending futures will be failed.
+     * Close listeners will be invoked as if connection is closed by server.
+     */
+    private void stop() {
+        close0(null);
+
+        if (reader != null) {
+            try {
+                U.interrupt(reader);
+                U.join(reader);
+
+                reader = null;
+            }
+            catch (IgniteInterruptedCheckedException ignored) {
+                Thread.currentThread().interrupt();
+
+                log.warn("Got interrupted while waiting for reader thread to shut down (will return).");
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addEventListener(HadoopIgfsIpcIoListener lsnr) {
+        if (!busyLock.readLock().tryLock()) {
+            lsnr.onClose();
+
+            return;
+        }
+
+        boolean invokeNow = false;
+
+        try {
+            invokeNow = stopping;
+
+            if (!invokeNow)
+                lsnrs.add(lsnr);
+        }
+        finally {
+            busyLock.readLock().unlock();
+
+            if (invokeNow)
+                lsnr.onClose();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeEventListener(HadoopIgfsIpcIoListener lsnr) {
+        lsnrs.remove(lsnr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException {
+        return send(msg, null, 0, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff,
+        int outLen) throws IgniteCheckedException {
+        assert outBuf == null || msg.command() == IgfsIpcCommand.READ_BLOCK;
+
+        if (!busyLock.readLock().tryLock())
+            throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
+                "closed).");
+
+        try {
+            if (stopping)
+                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
+                    "closed).");
+
+            long reqId = reqIdCnt.getAndIncrement();
+
+            HadoopIgfsFuture<T> fut = new HadoopIgfsFuture<>();
+
+            fut.outputBuffer(outBuf);
+            fut.outputOffset(outOff);
+            fut.outputLength(outLen);
+            fut.read(msg.command() == IgfsIpcCommand.READ_BLOCK);
+
+            HadoopIgfsFuture oldFut = reqMap.putIfAbsent(reqId, fut);
+
+            assert oldFut == null;
+
+            if (log.isDebugEnabled())
+                log.debug("Sending IGFS message [reqId=" + reqId + ", msg=" + msg + ']');
+
+            byte[] hdr = IgfsMarshaller.createHeader(reqId, msg.command());
+
+            IgniteCheckedException err = null;
+
+            try {
+                synchronized (this) {
+                    marsh.marshall(msg, hdr, out);
+
+                    out.flush(); // Blocking operation + sometimes system call.
+                }
+            }
+            catch (IgniteCheckedException e) {
+                err = e;
+            }
+            catch (IOException e) {
+                err = new HadoopIgfsCommunicationException(e);
+            }
+
+            if (err != null) {
+                reqMap.remove(reqId, fut);
+
+                fut.onDone(err);
+            }
+
+            return fut;
+        }
+        finally {
+            busyLock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendPlain(IgfsMessage msg) throws IgniteCheckedException {
+        if (!busyLock.readLock().tryLock())
+            throw new HadoopIgfsCommunicationException("Failed to send message (client is being " +
+                "concurrently closed).");
+
+        try {
+            if (stopping)
+                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently closed).");
+
+            assert msg.command() == IgfsIpcCommand.WRITE_BLOCK;
+
+            IgfsStreamControlRequest req = (IgfsStreamControlRequest)msg;
+
+            byte[] hdr = IgfsMarshaller.createHeader(-1, IgfsIpcCommand.WRITE_BLOCK);
+
+            U.longToBytes(req.streamId(), hdr, 12);
+            U.intToBytes(req.length(), hdr, 20);
+
+            synchronized (this) {
+                out.write(hdr);
+                out.write(req.data(), (int)req.position(), req.length());
+
+                out.flush();
+            }
+        }
+        catch (IOException e) {
+            throw new HadoopIgfsCommunicationException(e);
+        }
+        finally {
+            busyLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Closes client but does not wait.
+     *
+     * @param err Error.
+     */
+    private void close0(@Nullable Throwable err) {
+        busyLock.writeLock().lock();
+
+        try {
+            if (stopping)
+                return;
+
+            stopping = true;
+        }
+        finally {
+            busyLock.writeLock().unlock();
+        }
+
+        if (err == null)
+            err = new IgniteCheckedException("Failed to perform request (connection was concurrently closed before response " +
+                "is received).");
+
+        // Clean up resources.
+        U.closeQuiet(out);
+
+        if (endpoint != null)
+            endpoint.close();
+
+        // Unwind futures. We can safely iterate here because no more futures will be added.
+        Iterator<HadoopIgfsFuture> it = reqMap.values().iterator();
+
+        while (it.hasNext()) {
+            HadoopIgfsFuture fut = it.next();
+
+            fut.onDone(err);
+
+            it.remove();
+        }
+
+        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
+            lsnr.onClose();
+    }
+
+    /**
+     * Do not extend {@code GridThread} to minimize class dependencies.
+     */
+    private class ReaderThread extends Thread {
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void run() {
+            // Error to fail pending futures.
+            Throwable err = null;
+
+            try {
+                InputStream in = endpoint.inputStream();
+
+                IgfsDataInputStream dis = new IgfsDataInputStream(in);
+
+                byte[] hdr = new byte[IgfsMarshaller.HEADER_SIZE];
+                byte[] msgHdr = new byte[IgfsControlResponse.RES_HEADER_SIZE];
+
+                while (!Thread.currentThread().isInterrupted()) {
+                    dis.readFully(hdr);
+
+                    long reqId = U.bytesToLong(hdr, 0);
+
+                    // We don't wait for write responses, therefore reqId is -1.
+                    if (reqId == -1) {
+                        // We received a response which normally should not be sent. It must contain an error.
+                        dis.readFully(msgHdr);
+
+                        assert msgHdr[4] != 0;
+
+                        String errMsg = dis.readUTF();
+
+                        // Error code.
+                        dis.readInt();
+
+                        long streamId = dis.readLong();
+
+                        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
+                            lsnr.onError(streamId, errMsg);
+                    }
+                    else {
+                        HadoopIgfsFuture<Object> fut = reqMap.remove(reqId);
+
+                        if (fut == null) {
+                            String msg = "Failed to read response from server: response closure is unavailable for " +
+                                "requestId (will close connection):" + reqId;
+
+                            log.warn(msg);
+
+                            err = new IgniteCheckedException(msg);
+
+                            break;
+                        }
+                        else {
+                            try {
+                                IgfsIpcCommand cmd = IgfsIpcCommand.valueOf(U.bytesToInt(hdr, 8));
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Received IGFS response [reqId=" + reqId + ", cmd=" + cmd + ']');
+
+                                Object res = null;
+
+                                if (fut.read()) {
+                                    dis.readFully(msgHdr);
+
+                                    boolean hasErr = msgHdr[4] != 0;
+
+                                    if (hasErr) {
+                                        String errMsg = dis.readUTF();
+
+                                        // Error code.
+                                        Integer errCode = dis.readInt();
+
+                                        IgfsControlResponse.throwError(errCode, errMsg);
+                                    }
+
+                                    int blockLen = U.bytesToInt(msgHdr, 5);
+
+                                    int readLen = Math.min(blockLen, fut.outputLength());
+
+                                    if (readLen > 0) {
+                                        assert fut.outputBuffer() != null;
+
+                                        dis.readFully(fut.outputBuffer(), fut.outputOffset(), readLen);
+                                    }
+
+                                    if (readLen != blockLen) {
+                                        byte[] buf = new byte[blockLen - readLen];
+
+                                        dis.readFully(buf);
+
+                                        res = buf;
+                                    }
+                                }
+                                else
+                                    res = marsh.unmarshall(cmd, hdr, dis);
+
+                                fut.onDone(res);
+                            }
+                            catch (IgniteCheckedException e) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Failed to apply response closure (will fail request future): " +
+                                        e.getMessage());
+
+                                fut.onDone(e);
+
+                                err = e;
+                            }
+                        }
+                    }
+                }
+            }
+            catch (EOFException ignored) {
+                err = new IgniteCheckedException("Failed to read response from server (connection was closed by remote peer).");
+            }
+            catch (IOException e) {
+                if (!stopping)
+                    log.error("Failed to read data (connection will be closed)", e);
+
+                err = new HadoopIgfsCommunicationException(e);
+            }
+            catch (IgniteCheckedException e) {
+                if (!stopping)
+                    log.error("Failed to obtain endpoint input stream (connection will be closed)", e);
+
+                err = e;
+            }
+            finally {
+                close0(err);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return getClass().getSimpleName() + " [endpointAddr=" + endpointAddr + ", activeCnt=" + activeCnt +
+            ", stopping=" + stopping + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
index 10d764e..049e2b7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsIpcIoListener.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.igfs.hadoop;
 
 /**
- * Listens to the events of {@link HadoopIpcIgfsIo}.
+ * Listens to the events of {@link HadoopIgfsIpcIo}.
  */
 public interface HadoopIgfsIpcIoListener {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutProc.java
new file mode 100644
index 0000000..4cfacb9
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutProc.java
@@ -0,0 +1,466 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.*;
+
+/**
+ * Communication with external process (TCP or shmem).
+ */
+public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener {
+    /** Expected result is boolean. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Boolean> BOOL_RES = createClosure();
+
+    /** Expected result is boolean. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Long> LONG_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsFile> FILE_RES = createClosure();
+
+    /** Expected result is {@code IgfsHandshakeResponse} */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        IgfsHandshakeResponse> HANDSHAKE_RES = createClosure();
+
+    /** Expected result is {@code IgfsStatus} */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsStatus> STATUS_RES =
+        createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        IgfsInputStreamDescriptor> STREAM_DESCRIPTOR_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsFile>> FILE_COL_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsPath>> PATH_COL_RES = createClosure();
+
+    /** Expected result is {@code IgfsPathSummary}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsPathSummary> SUMMARY_RES =
+        createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsBlockLocation>> BLOCK_LOCATION_COL_RES = createClosure();
+
+    /** Grid name. */
+    private final String grid;
+
+    /** IGFS name. */
+    private final String igfs;
+
+    /** Client log. */
+    private final Log log;
+
+    /** Client IO. */
+    private final HadoopIgfsIpcIo io;
+
+    /** Event listeners. */
+    private final Map<Long, HadoopIgfsStreamEventListener> lsnrs = new ConcurrentHashMap8<>();
+
+    /**
+     * Constructor for TCP endpoint.
+     *
+     * @param host Host.
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    public HadoopIgfsOutProc(String host, int port, String grid, String igfs, Log log) throws IOException {
+        this(host, port, grid, igfs, false, log);
+    }
+
+    /**
+     * Constructor for shmem endpoint.
+     *
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    public HadoopIgfsOutProc(int port, String grid, String igfs, Log log) throws IOException {
+        this(null, port, grid, igfs, true, log);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param host Host.
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param shmem Shared memory flag.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    private HadoopIgfsOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log)
+        throws IOException {
+        assert host != null && !shmem || host == null && shmem :
+            "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
+
+        String endpoint = host != null ? host + ":" + port : "shmem:" + port;
+
+        this.grid = grid;
+        this.igfs = igfs;
+        this.log = log;
+
+        io = HadoopIgfsIpcIo.get(log, endpoint);
+
+        io.addEventListener(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException {
+        final IgfsHandshakeRequest req = new IgfsHandshakeRequest();
+
+        req.gridName(grid);
+        req.igfsName(igfs);
+        req.logDirectory(logDir);
+
+        return io.send(req).chain(HANDSHAKE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(boolean force) {
+        assert io != null;
+
+        io.removeEventListener(this);
+
+        if (force)
+            io.forceClose();
+        else
+            io.release();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(INFO);
+        msg.path(path);
+
+        return io.send(msg).chain(FILE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(UPDATE);
+        msg.path(path);
+        msg.properties(props);
+
+        return io.send(msg).chain(FILE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(SET_TIMES);
+        msg.path(path);
+        msg.accessTime(accessTime);
+        msg.modificationTime(modificationTime);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(RENAME);
+        msg.path(src);
+        msg.destinationPath(dest);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(DELETE);
+        msg.path(path);
+        msg.flag(recursive);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
+        throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(AFFINITY);
+        msg.path(path);
+        msg.start(start);
+        msg.length(len);
+
+        return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(PATH_SUMMARY);
+        msg.path(path);
+
+        return io.send(msg).chain(SUMMARY_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(MAKE_DIRECTORIES);
+        msg.path(path);
+        msg.properties(props);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(LIST_FILES);
+        msg.path(path);
+
+        return io.send(msg).chain(FILE_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(LIST_PATHS);
+        msg.path(path);
+
+        return io.send(msg).chain(PATH_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
+        return io.send(new IgfsStatusRequest()).chain(STATUS_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_READ);
+        msg.path(path);
+        msg.flag(false);
+
+        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path,
+        int seqReadsBeforePrefetch) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_READ);
+        msg.path(path);
+        msg.flag(true);
+        msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
+
+        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
+        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_CREATE);
+        msg.path(path);
+        msg.flag(overwrite);
+        msg.colocate(colocate);
+        msg.properties(props);
+        msg.replication(replication);
+        msg.blockSize(blockSize);
+
+        Long streamId = io.send(msg).chain(LONG_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, streamId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
+        @Nullable Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_APPEND);
+        msg.path(path);
+        msg.flag(create);
+        msg.properties(props);
+
+        Long streamId = io.send(msg).chain(LONG_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, streamId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPlainFuture<byte[]> readData(HadoopIgfsStreamDelegate desc, long pos, int len,
+        final @Nullable byte[] outBuf, final int outOff, final int outLen) {
+        assert len > 0;
+
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(READ_BLOCK);
+        msg.streamId((long) desc.target());
+        msg.position(pos);
+        msg.length(len);
+
+        try {
+            return io.send(msg, outBuf, outOff, outLen);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridPlainFutureAdapter<>(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeData(HadoopIgfsStreamDelegate desc, byte[] data, int off, int len)
+        throws IOException {
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(WRITE_BLOCK);
+        msg.streamId((long) desc.target());
+        msg.data(data);
+        msg.position(off);
+        msg.length(len);
+
+        try {
+            io.sendPlain(msg);
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush(HadoopIgfsStreamDelegate delegate) throws IOException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeStream(HadoopIgfsStreamDelegate desc) throws IOException {
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(CLOSE);
+        msg.streamId((long)desc.target());
+
+        try {
+            io.send(msg).chain(BOOL_RES).get();
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addEventListener(HadoopIgfsStreamDelegate desc,
+        HadoopIgfsStreamEventListener lsnr) {
+        long streamId = desc.target();
+
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.put(streamId, lsnr);
+
+        assert lsnr0 == null || lsnr0 == lsnr;
+
+        if (log.isDebugEnabled())
+            log.debug("Added stream event listener [streamId=" + streamId + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeEventListener(HadoopIgfsStreamDelegate desc) {
+        long streamId = desc.target();
+
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.remove(streamId);
+
+        if (lsnr0 != null && log.isDebugEnabled())
+            log.debug("Removed stream event listener [streamId=" + streamId + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        for (HadoopIgfsStreamEventListener lsnr : lsnrs.values()) {
+            try {
+                lsnr.onClose();
+            }
+            catch (IgniteCheckedException e) {
+                log.warn("Got exception from stream event listener (will ignore): " + lsnr, e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(long streamId, String errMsg) {
+        HadoopIgfsStreamEventListener lsnr = lsnrs.get(streamId);
+
+        if (lsnr != null)
+            lsnr.onError(errMsg);
+        else
+            log.warn("Received write error response for not registered output stream (will ignore) " +
+                "[streamId= " + streamId + ']');
+    }
+
+    /**
+     * Creates conversion closure for given type.
+     *
+     * @param <T> Type of expected result.
+     * @return Conversion closure.
+     */
+    @SuppressWarnings("unchecked")
+    private static <T> GridPlainClosure<GridPlainFuture<IgfsMessage>, T> createClosure() {
+        return new GridPlainClosure<GridPlainFuture<IgfsMessage>, T>() {
+            @Override public T apply(GridPlainFuture<IgfsMessage> fut) throws IgniteCheckedException {
+                IgfsControlResponse res = (IgfsControlResponse)fut.get();
+
+                if (res.hasError())
+                    res.throwError();
+
+                return (T)res.response();
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutputStream.java
new file mode 100644
index 0000000..9e08bdb
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsOutputStream.java
@@ -0,0 +1,201 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * IGFS Hadoop output stream implementation.
+ */
+public class HadoopIgfsOutputStream extends OutputStream implements HadoopIgfsStreamEventListener {
+    /** Log instance. */
+    private Log log;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Log stream ID. */
+    private long logStreamId;
+
+    /** Server stream delegate. */
+    private HadoopIgfsStreamDelegate delegate;
+
+    /** Closed flag. */
+    private volatile boolean closed;
+
+    /** Flag set if stream was closed due to connection breakage. */
+    private boolean connBroken;
+
+    /** Error message. */
+    private volatile String errMsg;
+
+    /** Read time. */
+    private long writeTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of written bytes. */
+    private long total;
+
+    /**
+     * Creates light output stream.
+     *
+     * @param delegate Server stream delegate.
+     * @param log Logger to use.
+     * @param clientLog Client logger.
+     */
+    public HadoopIgfsOutputStream(HadoopIgfsStreamDelegate delegate, Log log,
+        IgfsLogger clientLog, long logStreamId) {
+        this.delegate = delegate;
+        this.log = log;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        lastTs = System.nanoTime();
+
+        delegate.hadoop().addEventListener(delegate, this);
+    }
+
+    /**
+     * Read start.
+     */
+    private void writeStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void writeEnd() {
+        long now = System.nanoTime();
+
+        writeTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
+        check();
+
+        writeStart();
+
+        try {
+            delegate.hadoop().writeData(delegate, b, off, len);
+
+            total += len;
+        }
+        finally {
+            writeEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(int b) throws IOException {
+        write(new byte[] {(byte)b});
+
+        total++;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush() throws IOException {
+        delegate.hadoop().flush(delegate);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        if (!closed) {
+            if (log.isDebugEnabled())
+                log.debug("Closing output stream: " + delegate);
+
+            writeStart();
+
+            delegate.hadoop().closeStream(delegate);
+
+            markClosed(false);
+
+            writeEnd();
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
+
+            if (log.isDebugEnabled())
+                log.debug("Closed output stream [delegate=" + delegate + ", writeTime=" + writeTime / 1000 +
+                    ", userTime=" + userTime / 1000 + ']');
+        }
+        else if(connBroken)
+            throw new IOException(
+                "Failed to close stream, because connection was broken (data could have been lost).");
+    }
+
+    /**
+     * Marks stream as closed.
+     *
+     * @param connBroken {@code True} if connection with server was lost.
+     */
+    private void markClosed(boolean connBroken) {
+        // It is ok to have race here.
+        if (!closed) {
+            closed = true;
+
+            delegate.hadoop().removeEventListener(delegate);
+
+            this.connBroken = connBroken;
+        }
+    }
+
+    /**
+     * @throws IOException If check failed.
+     */
+    private void check() throws IOException {
+        String errMsg0 = errMsg;
+
+        if (errMsg0 != null)
+            throw new IOException(errMsg0);
+
+        if (closed) {
+            if (connBroken)
+                throw new IOException("Server connection was lost.");
+            else
+                throw new IOException("Stream is closed.");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() throws IgniteCheckedException {
+        markClosed(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(String errMsg) {
+        this.errMsg = errMsg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
index 94a4449..69e6503 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
@@ -384,7 +384,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopOutProcIgfs(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
@@ -408,7 +408,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopOutProcIgfs(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
+                    hadoop = new HadoopIgfsOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
                         log);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
@@ -430,7 +430,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             HadoopIgfsEx hadoop = null;
 
             try {
-                hadoop = new HadoopOutProcIgfs(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                hadoop = new HadoopIgfsOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
 
                 curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
deleted file mode 100644
index 5a008bd..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
+++ /dev/null
@@ -1,626 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * IGFS input stream wrapper for hadoop interfaces.
- */
-@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-public final class HadoopInputIgfsStream extends InputStream implements Seekable, PositionedReadable,
-    HadoopIgfsStreamEventListener {
-    /** Minimum buffer size. */
-    private static final int MIN_BUF_SIZE = 4 * 1024;
-
-    /** Server stream delegate. */
-    private HadoopIgfsStreamDelegate delegate;
-
-    /** Stream ID used by logger. */
-    private long logStreamId;
-
-    /** Stream position. */
-    private long pos;
-
-    /** Stream read limit. */
-    private long limit;
-
-    /** Mark position. */
-    private long markPos = -1;
-
-    /** Prefetch buffer. */
-    private DoubleFetchBuffer buf = new DoubleFetchBuffer();
-
-    /** Buffer half size for double-buffering. */
-    private int bufHalfSize;
-
-    /** Closed flag. */
-    private volatile boolean closed;
-
-    /** Flag set if stream was closed due to connection breakage. */
-    private boolean connBroken;
-
-    /** Logger. */
-    private Log log;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Read time. */
-    private long readTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of read bytes. */
-    private long total;
-
-    /**
-     * Creates input stream.
-     *
-     * @param delegate Server stream delegate.
-     * @param limit Read limit.
-     * @param bufSize Buffer size.
-     * @param log Log.
-     * @param clientLog Client logger.
-     */
-    public HadoopInputIgfsStream(HadoopIgfsStreamDelegate delegate, long limit, int bufSize, Log log,
-        IgfsLogger clientLog, long logStreamId) {
-        assert limit >= 0;
-
-        this.delegate = delegate;
-        this.limit = limit;
-        this.log = log;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        bufHalfSize = Math.max(bufSize, MIN_BUF_SIZE);
-
-        lastTs = System.nanoTime();
-
-        delegate.hadoop().addEventListener(delegate, this);
-    }
-
-    /**
-     * Read start.
-     */
-    private void readStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void readEnd() {
-        long now = System.nanoTime();
-
-        readTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read() throws IOException {
-        checkClosed();
-
-        readStart();
-
-        try {
-            if (eof())
-                return -1;
-
-            buf.refreshAhead(pos);
-
-            int res = buf.atPosition(pos);
-
-            pos++;
-            total++;
-
-            buf.refreshAhead(pos);
-
-            return res;
-        }
-        catch (IgniteCheckedException e) {
-            throw HadoopIgfsUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(@NotNull byte[] b, int off, int len) throws IOException {
-        checkClosed();
-
-        if (eof())
-            return -1;
-
-        readStart();
-
-        try {
-            long remaining = limit - pos;
-
-            int read = buf.flatten(b, pos, off, len);
-
-            pos += read;
-            total += read;
-            remaining -= read;
-
-            if (remaining > 0 && read != len) {
-                int readAmt = (int)Math.min(remaining, len - read);
-
-                delegate.hadoop().readData(delegate, pos, readAmt, b, off + read, len - read).get();
-
-                read += readAmt;
-                pos += readAmt;
-                total += readAmt;
-            }
-
-            buf.refreshAhead(pos);
-
-            return read;
-        }
-        catch (IgniteCheckedException e) {
-            throw HadoopIgfsUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long skip(long n) throws IOException {
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSkip(logStreamId, n);
-
-        long oldPos = pos;
-
-        if (pos + n <= limit)
-            pos += n;
-        else
-            pos = limit;
-
-        buf.refreshAhead(pos);
-
-        return pos - oldPos;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int available() throws IOException {
-        checkClosed();
-
-        int available = buf.available(pos);
-
-        assert available >= 0;
-
-        return available;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void close() throws IOException {
-        if (!closed) {
-            readStart();
-
-            if (log.isDebugEnabled())
-                log.debug("Closing input stream: " + delegate);
-
-            delegate.hadoop().closeStream(delegate);
-
-            readEnd();
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
-
-            markClosed(false);
-
-            if (log.isDebugEnabled())
-                log.debug("Closed stream [delegate=" + delegate + ", readTime=" + readTime +
-                    ", userTime=" + userTime + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void mark(int readLimit) {
-        markPos = pos;
-
-        if (clientLog.isLogEnabled())
-            clientLog.logMark(logStreamId, readLimit);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void reset() throws IOException {
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logReset(logStreamId);
-
-        if (markPos == -1)
-            throw new IOException("Stream was not marked.");
-
-        pos = markPos;
-
-        buf.refreshAhead(pos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean markSupported() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(long position, byte[] buf, int off, int len) throws IOException {
-        long remaining = limit - position;
-
-        int read = (int)Math.min(len, remaining);
-
-        // Return -1 at EOF.
-        if (read == 0)
-            return -1;
-
-        readFully(position, buf, off, read);
-
-        return read;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void readFully(long position, byte[] buf, int off, int len) throws IOException {
-        long remaining = limit - position;
-
-        checkClosed();
-
-        if (len > remaining)
-            throw new EOFException("End of stream reached before data was fully read.");
-
-        readStart();
-
-        try {
-            int read = this.buf.flatten(buf, position, off, len);
-
-            total += read;
-
-            if (read != len) {
-                int readAmt = len - read;
-
-                delegate.hadoop().readData(delegate, position + read, readAmt, buf, off + read, readAmt).get();
-
-                total += readAmt;
-            }
-
-            if (clientLog.isLogEnabled())
-                clientLog.logRandomRead(logStreamId, position, len);
-        }
-        catch (IgniteCheckedException e) {
-            throw HadoopIgfsUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(long position, byte[] buf) throws IOException {
-        readFully(position, buf, 0, buf.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void seek(long pos) throws IOException {
-        A.ensure(pos >= 0, "position must be non-negative");
-
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSeek(logStreamId, pos);
-
-        if (pos > limit)
-            pos = limit;
-
-        if (log.isDebugEnabled())
-            log.debug("Seek to position [delegate=" + delegate + ", pos=" + pos + ", oldPos=" + this.pos + ']');
-
-        this.pos = pos;
-
-        buf.refreshAhead(pos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long getPos() {
-        return pos;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized boolean seekToNewSource(long targetPos) {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() {
-        markClosed(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(String errMsg) {
-        // No-op.
-    }
-
-    /**
-     * Marks stream as closed.
-     *
-     * @param connBroken {@code True} if connection with server was lost.
-     */
-    private void markClosed(boolean connBroken) {
-        // It is ok to have race here.
-        if (!closed) {
-            closed = true;
-
-            this.connBroken = connBroken;
-
-            delegate.hadoop().removeEventListener(delegate);
-        }
-    }
-
-    /**
-     * @throws IOException If check failed.
-     */
-    private void checkClosed() throws IOException {
-        if (closed) {
-            if (connBroken)
-                throw new IOException("Server connection was lost.");
-            else
-                throw new IOException("Stream is closed.");
-        }
-    }
-
-    /**
-     * @return {@code True} if end of stream reached.
-     */
-    private boolean eof() {
-        return limit == pos;
-    }
-
-    /**
-     * Asynchronous prefetch buffer.
-     */
-    private static class FetchBufferPart {
-        /** Read future. */
-        private GridPlainFuture<byte[]> readFut;
-
-        /** Position of cached chunk in file. */
-        private long pos;
-
-        /** Prefetch length. Need to store as read future result might be not available yet. */
-        private int len;
-
-        /**
-         * Creates fetch buffer part.
-         *
-         * @param readFut Read future for this buffer.
-         * @param pos Read position.
-         * @param len Chunk length.
-         */
-        private FetchBufferPart(GridPlainFuture<byte[]> readFut, long pos, int len) {
-            this.readFut = readFut;
-            this.pos = pos;
-            this.len = len;
-        }
-
-        /**
-         * Copies cached data if specified position matches cached region.
-         *
-         * @param dst Destination buffer.
-         * @param pos Read position in file.
-         * @param dstOff Offset in destination buffer from which start writing.
-         * @param len Maximum number of bytes to copy.
-         * @return Number of bytes copied.
-         * @throws IgniteCheckedException If read future failed.
-         */
-        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
-            // If read start position is within cached boundaries.
-            if (contains(pos)) {
-                byte[] data = readFut.get();
-
-                int srcPos = (int)(pos - this.pos);
-                int cpLen = Math.min(len, data.length - srcPos);
-
-                U.arrayCopy(data, srcPos, dst, dstOff, cpLen);
-
-                return cpLen;
-            }
-
-            return 0;
-        }
-
-        /**
-         * @return {@code True} if data is ready to be read.
-         */
-        public boolean ready() {
-            return readFut.isDone();
-        }
-
-        /**
-         * Checks if current buffer part contains given position.
-         *
-         * @param pos Position to check.
-         * @return {@code True} if position matches buffer region.
-         */
-        public boolean contains(long pos) {
-            return this.pos <= pos && this.pos + len > pos;
-        }
-    }
-
-    private class DoubleFetchBuffer {
-        /**  */
-        private FetchBufferPart first;
-
-        /** */
-        private FetchBufferPart second;
-
-        /**
-         * Copies fetched data from both buffers to destination array if cached region matched read position.
-         *
-         * @param dst Destination buffer.
-         * @param pos Read position in file.
-         * @param dstOff Destination buffer offset.
-         * @param len Maximum number of bytes to copy.
-         * @return Number of bytes copied.
-         * @throws IgniteCheckedException If any read operation failed.
-         */
-        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
-            assert dstOff >= 0;
-            assert dstOff + len <= dst.length : "Invalid indices [dst.length=" + dst.length + ", dstOff=" + dstOff +
-                ", len=" + len + ']';
-
-            int bytesCopied = 0;
-
-            if (first != null) {
-                bytesCopied += first.flatten(dst, pos, dstOff, len);
-
-                if (bytesCopied != len && second != null) {
-                    assert second.pos == first.pos + first.len;
-
-                    bytesCopied += second.flatten(dst, pos + bytesCopied, dstOff + bytesCopied, len - bytesCopied);
-                }
-            }
-
-            return bytesCopied;
-        }
-
-        /**
-         * Gets byte at specified position in buffer.
-         *
-         * @param pos Stream position.
-         * @return Read byte.
-         * @throws IgniteCheckedException If read failed.
-         */
-        public int atPosition(long pos) throws IgniteCheckedException {
-            // Should not reach here if stream contains no data.
-            assert first != null;
-
-            if (first.contains(pos)) {
-                byte[] bytes = first.readFut.get();
-
-                return bytes[((int)(pos - first.pos))] & 0xFF;
-            }
-            else {
-                assert second != null;
-                assert second.contains(pos);
-
-                byte[] bytes = second.readFut.get();
-
-                return bytes[((int)(pos - second.pos))] & 0xFF;
-            }
-        }
-
-        /**
-         * Starts asynchronous buffer refresh if needed, depending on current position.
-         *
-         * @param pos Current stream position.
-         */
-        public void refreshAhead(long pos) {
-            if (fullPrefetch(pos)) {
-                first = fetch(pos, bufHalfSize);
-                second = fetch(pos + bufHalfSize, bufHalfSize);
-            }
-            else if (needFlip(pos)) {
-                first = second;
-
-                second = fetch(first.pos + first.len, bufHalfSize);
-            }
-        }
-
-        /**
-         * @param pos Position from which read is expected.
-         * @return Number of bytes available to be read without blocking.
-         */
-        public int available(long pos) {
-            int available = 0;
-
-            if (first != null) {
-                if (first.contains(pos)) {
-                    if (first.ready()) {
-                        available += (pos - first.pos);
-
-                        if (second != null && second.ready())
-                            available += second.len;
-                    }
-                }
-                else {
-                    if (second != null && second.contains(pos) && second.ready())
-                        available += (pos - second.pos);
-                }
-            }
-
-            return available;
-        }
-
-        /**
-         * Checks if position shifted enough to forget previous buffer.
-         *
-         * @param pos Current position.
-         * @return {@code True} if need flip buffers.
-         */
-        private boolean needFlip(long pos) {
-            // Return true if we read more then half of second buffer.
-            return second != null && second.contains(pos);
-        }
-
-        /**
-         * Determines if all cached bytes should be discarded and new region should be
-         * prefetched.
-         *
-         * @param curPos Current stream position.
-         * @return {@code True} if need to refresh both blocks.
-         */
-        private boolean fullPrefetch(long curPos) {
-            // If no data was prefetched yet, return true.
-            return first == null || curPos < first.pos || (second != null && curPos >= second.pos + second.len);
-        }
-
-        /**
-         * Starts asynchronous fetch for given region.
-         *
-         * @param pos Position to read from.
-         * @param size Number of bytes to read.
-         * @return Fetch buffer part.
-         */
-        private FetchBufferPart fetch(long pos, int size) {
-            long remaining = limit - pos;
-
-            size = (int)Math.min(size, remaining);
-
-            return size <= 0 ? null :
-                new FetchBufferPart(delegate.hadoop().readData(delegate, pos, size, null, 0, 0), pos, size);
-        }
-    }
-}


[23/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemLoopbackPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemLoopbackPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemLoopbackPrimarySelfTest.java
deleted file mode 100644
index 47e1c86..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemLoopbackPrimarySelfTest.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.igfs;
-
-import java.util.*;
-
-import static org.apache.ignite.igfs.IgfsMode.*;
-import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
-
-/**
- * Tests Hadoop 2.x file system in primary mode.
- */
-public class IgfsHadoop20FileSystemLoopbackPrimarySelfTest extends IgfsHadoop20FileSystemAbstractSelfTest {
-    /**
-     * Creates test in primary mode.
-     */
-    public IgfsHadoop20FileSystemLoopbackPrimarySelfTest() {
-        super(PRIMARY);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String primaryFileSystemUriPath() {
-        return "igfs://igfs:" + getTestGridName(0) + "@/";
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String primaryFileSystemConfigPath() {
-        return "/modules/core/src/test/config/hadoop/core-site-loopback.xml";
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Map<String, String> primaryIpcEndpointConfiguration(final String gridName) {
-        return new HashMap<String, String>() {{
-            put("type", "tcp");
-            put("port", String.valueOf(DFLT_IPC_PORT + getTestGridIndex(gridName)));
-        }};
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String secondaryFileSystemUriPath() {
-        assert false;
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String secondaryFileSystemConfigPath() {
-        assert false;
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Map<String, String> secondaryIpcEndpointConfiguration() {
-        assert false;
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemShmemPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemShmemPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemShmemPrimarySelfTest.java
deleted file mode 100644
index 631f188..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemShmemPrimarySelfTest.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.igfs;
-
-import java.util.*;
-
-import static org.apache.ignite.igfs.IgfsMode.*;
-import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
-
-/**
- * Tests Hadoop 2.x file system in primary mode.
- */
-public class IgfsHadoop20FileSystemShmemPrimarySelfTest extends IgfsHadoop20FileSystemAbstractSelfTest {
-    /**
-     * Creates test in primary mode.
-     */
-    public IgfsHadoop20FileSystemShmemPrimarySelfTest() {
-        super(PRIMARY);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String primaryFileSystemUriPath() {
-        return "igfs://igfs:" + getTestGridName(0) + "@/";
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String primaryFileSystemConfigPath() {
-        return "/modules/core/src/test/config/hadoop/core-site.xml";
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Map<String, String> primaryIpcEndpointConfiguration(final String gridName) {
-        return new HashMap<String, String>() {{
-            put("type", "shmem");
-            put("port", String.valueOf(DFLT_IPC_PORT + getTestGridIndex(gridName)));
-        }};
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String secondaryFileSystemUriPath() {
-        assert false;
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected String secondaryFileSystemConfigPath() {
-        assert false;
-
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Map<String, String> secondaryIpcEndpointConfiguration() {
-        assert false;
-
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopDualAbstractSelfTest.java
deleted file mode 100644
index 2d29800..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoopDualAbstractSelfTest.java
+++ /dev/null
@@ -1,304 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.hadoop.fs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.igfs.IgfsMode.*;
-import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
-import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.*;
-
-/**
- * Tests for IGFS working in mode when remote file system exists: DUAL_SYNC, DUAL_ASYNC.
- */
-public abstract class IgfsHadoopDualAbstractSelfTest extends IgfsCommonAbstractTest {
-    /** IGFS block size. */
-    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
-
-    /** Amount of blocks to prefetch. */
-    protected static final int PREFETCH_BLOCKS = 1;
-
-    /** Amount of sequential block reads before prefetch is triggered. */
-    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
-
-    /** Secondary file system URI. */
-    protected static final String SECONDARY_URI = "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/";
-
-    /** Secondary file system configuration path. */
-    protected static final String SECONDARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml";
-
-    /** Primary file system URI. */
-    protected static final String PRIMARY_URI = "igfs://igfs:grid@/";
-
-    /** Primary file system configuration path. */
-    protected static final String PRIMARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback.xml";
-
-    /** Primary file system REST endpoint configuration map. */
-    protected static final Map<String, String> PRIMARY_REST_CFG = new HashMap<String, String>() {{
-        put("type", "tcp");
-        put("port", "10500");
-    }};
-
-    /** Secondary file system REST endpoint configuration map. */
-    protected static final Map<String, String> SECONDARY_REST_CFG = new HashMap<String, String>() {{
-        put("type", "tcp");
-        put("port", "11500");
-    }};
-
-    /** Directory. */
-    protected static final IgfsPath DIR = new IgfsPath("/dir");
-
-    /** Sub-directory. */
-    protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir");
-
-    /** File. */
-    protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file");
-
-    /** Default data chunk (128 bytes). */
-    protected static byte[] chunk;
-
-    /** Primary IGFS. */
-    protected static IgfsImpl igfs;
-
-    /** Secondary IGFS. */
-    protected static IgfsImpl igfsSecondary;
-
-    /** IGFS mode. */
-    protected final IgfsMode mode;
-
-    /**
-     * Constructor.
-     *
-     * @param mode IGFS mode.
-     */
-    protected IgfsHadoopDualAbstractSelfTest(IgfsMode mode) {
-        this.mode = mode;
-        assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
-    }
-
-    /**
-     * Start grid with IGFS.
-     *
-     * @param gridName Grid name.
-     * @param igfsName IGFS name
-     * @param mode IGFS mode.
-     * @param secondaryFs Secondary file system (optional).
-     * @param restCfg Rest configuration string (optional).
-     * @return Started grid instance.
-     * @throws Exception If failed.
-     */
-    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable Igfs secondaryFs, @Nullable Map<String, String> restCfg) throws Exception {
-        IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-        igfsCfg.setDataCacheName("dataCache");
-        igfsCfg.setMetaCacheName("metaCache");
-        igfsCfg.setName(igfsName);
-        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
-        igfsCfg.setDefaultMode(mode);
-        igfsCfg.setIpcEndpointConfiguration(restCfg);
-        igfsCfg.setSecondaryFileSystem(secondaryFs);
-        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
-        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
-
-        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
-
-        dataCacheCfg.setName("dataCache");
-        dataCacheCfg.setCacheMode(PARTITIONED);
-        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
-        dataCacheCfg.setBackups(0);
-        dataCacheCfg.setQueryIndexEnabled(false);
-        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
-        dataCacheCfg.setOffHeapMaxMemory(0);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("metaCache");
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        metaCacheCfg.setQueryIndexEnabled(false);
-        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        IgniteConfiguration cfg = new IgniteConfiguration();
-
-        cfg.setGridName(gridName);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
-        cfg.setIgfsConfiguration(igfsCfg);
-
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setConnectorConfiguration(null);
-
-        return G.start(cfg);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        chunk = new byte[128];
-
-        for (int i = 0; i < chunk.length; i++)
-            chunk[i] = (byte)i;
-
-        Ignite igniteSecondary = startGridWithIgfs("grid-secondary", "igfs-secondary", PRIMARY, null, SECONDARY_REST_CFG);
-
-        Igfs hadoopFs = new IgniteHadoopSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG);
-
-        Ignite ignite = startGridWithIgfs("grid", "igfs", mode, hadoopFs, PRIMARY_REST_CFG);
-
-        igfsSecondary = (IgfsImpl) igniteSecondary.fileSystem("igfs-secondary");
-        igfs = (IgfsImpl) ignite.fileSystem("igfs");
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        clear(igfs);
-        clear(igfsSecondary);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        G.stopAll(true);
-    }
-
-    /**
-     * Convenient method to group paths.
-     *
-     * @param paths Paths to group.
-     * @return Paths as array.
-     */
-    protected IgfsPath[] paths(IgfsPath... paths) {
-        return paths;
-    }
-
-    /**
-     * Check how prefetch override works.
-     *
-     * @throws Exception IF failed.
-     */
-    public void testOpenPrefetchOverride() throws Exception {
-        create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE));
-
-        // Write enough data to the secondary file system.
-        final int blockSize = IGFS_BLOCK_SIZE;
-
-        IgfsOutputStream out = igfsSecondary.append(FILE, false);
-
-        int totalWritten = 0;
-
-        while (totalWritten < blockSize * 2 + chunk.length) {
-            out.write(chunk);
-
-            totalWritten += chunk.length;
-        }
-
-        out.close();
-
-        awaitFileClose(igfsSecondary, FILE);
-
-        // Instantiate file system with overridden "seq reads before prefetch" property.
-        Configuration cfg = new Configuration();
-
-        cfg.addResource(U.resolveIgniteUrl(PRIMARY_CFG));
-
-        int seqReads = SEQ_READS_BEFORE_PREFETCH + 1;
-
-        cfg.setInt(String.format(PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, "igfs:grid@"), seqReads);
-
-        FileSystem fs = FileSystem.get(new URI(PRIMARY_URI), cfg);
-
-        // Read the first two blocks.
-        Path fsHome = new Path(PRIMARY_URI);
-        Path dir = new Path(fsHome, DIR.name());
-        Path subdir = new Path(dir, SUBDIR.name());
-        Path file = new Path(subdir, FILE.name());
-
-        FSDataInputStream fsIn = fs.open(file);
-
-        final byte[] readBuf = new byte[blockSize * 2];
-
-        fsIn.readFully(0, readBuf, 0, readBuf.length);
-
-        // Wait for a while for prefetch to finish (if any).
-        IgfsMetaManager meta = igfs.context().meta();
-
-        IgfsFileInfo info = meta.info(meta.fileId(FILE));
-
-        IgfsBlockKey key = new IgfsBlockKey(info.id(), info.affinityKey(), info.evictExclude(), 2);
-
-        GridCache<IgfsBlockKey, byte[]> dataCache = igfs.context().kernalContext().cache().cache(
-            igfs.configuration().getDataCacheName());
-
-        for (int i = 0; i < 10; i++) {
-            if (dataCache.containsKey(key))
-                break;
-            else
-                U.sleep(100);
-        }
-
-        fsIn.close();
-
-        // Remove the file from the secondary file system.
-        igfsSecondary.delete(FILE, false);
-
-        // Try reading the third block. Should fail.
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                IgfsInputStream in0 = igfs.open(FILE);
-
-                in0.seek(blockSize * 2);
-
-                try {
-                    in0.read(readBuf);
-                }
-                finally {
-                    U.closeQuiet(in0);
-                }
-
-                return null;
-            }
-        }, IOException.class,
-            "Failed to read data due to secondary file system exception: /dir/subdir/file");
-    }
-}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/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 ab80589..98817ee 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
@@ -410,17 +410,17 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
      * @throws Exception If failed.
      */
     public void testIpcCache() throws Exception {
-        IgfsHadoopEx hadoop = GridTestUtils.getFieldValue(fs, "rmtClient", "delegateRef", "value", "hadoop");
+        HadoopIgfsEx hadoop = GridTestUtils.getFieldValue(fs, "rmtClient", "delegateRef", "value", "hadoop");
 
-        if (hadoop instanceof IgfsHadoopOutProc) {
+        if (hadoop instanceof HadoopOutProcIgfs) {
             FileSystem fsOther = null;
 
             try {
-                Field field = IgfsHadoopIpcIo.class.getDeclaredField("ipcCache");
+                Field field = HadoopIpcIgfsIo.class.getDeclaredField("ipcCache");
 
                 field.setAccessible(true);
 
-                Map<String, IgfsHadoopIpcIo> cache = (Map<String, IgfsHadoopIpcIo>)field.get(null);
+                Map<String, HadoopIpcIgfsIo> cache = (Map<String, HadoopIpcIgfsIo>)field.get(null);
 
                 Configuration cfg = configuration(PRIMARY_AUTHORITY, skipEmbed, skipLocShmem);
 
@@ -441,13 +441,13 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
                 assertEquals(initSize, cache.size());
 
-                Field stopField = IgfsHadoopIpcIo.class.getDeclaredField("stopping");
+                Field stopField = HadoopIpcIgfsIo.class.getDeclaredField("stopping");
 
                 stopField.setAccessible(true);
 
-                IgfsHadoopIpcIo io = null;
+                HadoopIpcIgfsIo io = null;
 
-                for (Map.Entry<String, IgfsHadoopIpcIo> ioEntry : cache.entrySet()) {
+                for (Map.Entry<String, HadoopIpcIgfsIo> ioEntry : cache.entrySet()) {
                     if (endpoint.contains(ioEntry.getKey())) {
                         io = ioEntry.getValue();
 
@@ -2356,10 +2356,10 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
         cfg.setBoolean("fs.igfs.impl.disable.cache", true);
 
         if (skipEmbed)
-            cfg.setBoolean(String.format(IgfsHadoopUtils.PARAM_IGFS_ENDPOINT_NO_EMBED, authority), true);
+            cfg.setBoolean(String.format(HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED, authority), true);
 
         if (skipLocShmem)
-            cfg.setBoolean(String.format(IgfsHadoopUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority), true);
+            cfg.setBoolean(String.format(HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority), true);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
index 52cfbd9..0025da3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
@@ -125,15 +125,15 @@ public class IgniteHadoopFileSystemClientSelfTest extends IgfsCommonAbstractTest
         try {
             switchHandlerErrorFlag(true);
 
-            IgfsHadoop client = new IgfsHadoopOutProc("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG);
+            HadoopIgfs client = new HadoopOutProcIgfs("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG);
 
             client.handshake(null);
 
             IgfsPath path = new IgfsPath("/test1.file");
 
-            IgfsHadoopStreamDelegate delegate = client.create(path, true, false, 1, 1024, null);
+            HadoopIgfsStreamDelegate delegate = client.create(path, true, false, 1, 1024, null);
 
-            final IgfsHadoopOutputStream igfsOut = new IgfsHadoopOutputStream(delegate, LOG,
+            final HadoopOutputIgfsStream igfsOut = new HadoopOutputIgfsStream(delegate, LOG,
                 IgfsLogger.disabledLogger(), 0);
 
             // This call should return fine as exception is thrown for the first time.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
index 0eef7e1..9cc8868 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemHandshakeSelfTest.java
@@ -41,7 +41,7 @@ import static org.apache.ignite.cache.CacheDistributionMode.*;
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
+import static org.apache.ignite.internal.igfs.hadoop.HadoopIgfsUtils.*;
 import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
index bdbff8b..8172935 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
@@ -141,15 +141,15 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
      */
     @SuppressWarnings("unchecked")
     public void testIpcCache() throws Exception {
-        Field cacheField = IgfsHadoopIpcIo.class.getDeclaredField("ipcCache");
+        Field cacheField = HadoopIpcIgfsIo.class.getDeclaredField("ipcCache");
 
         cacheField.setAccessible(true);
 
-        Field activeCntField = IgfsHadoopIpcIo.class.getDeclaredField("activeCnt");
+        Field activeCntField = HadoopIpcIgfsIo.class.getDeclaredField("activeCnt");
 
         activeCntField.setAccessible(true);
 
-        Map<String, IgfsHadoopIpcIo> cache = (Map<String, IgfsHadoopIpcIo>)cacheField.get(null);
+        Map<String, HadoopIpcIgfsIo> cache = (Map<String, HadoopIpcIgfsIo>)cacheField.get(null);
 
         String name = "igfs:" + getTestGridName(0) + "@";
 
@@ -157,14 +157,14 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
 
         cfg.addResource(U.resolveIgniteUrl(HADOOP_FS_CFG));
         cfg.setBoolean("fs.igfs.impl.disable.cache", true);
-        cfg.setBoolean(String.format(IgfsHadoopUtils.PARAM_IGFS_ENDPOINT_NO_EMBED, name), true);
+        cfg.setBoolean(String.format(HadoopIgfsUtils.PARAM_IGFS_ENDPOINT_NO_EMBED, name), true);
 
         // Ensure that existing IO is reused.
         FileSystem fs1 = FileSystem.get(new URI("igfs://" + name + "/"), cfg);
 
         assertEquals(1, cache.size());
 
-        IgfsHadoopIpcIo io = null;
+        HadoopIpcIgfsIo io = null;
 
         System.out.println("CACHE: " + cache);
 
@@ -191,7 +191,7 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
         assertEquals(1, cache.size());
         assertEquals(1, ((AtomicInteger)activeCntField.get(io)).get());
 
-        Field stopField = IgfsHadoopIpcIo.class.getDeclaredField("stopping");
+        Field stopField = HadoopIpcIgfsIo.class.getDeclaredField("stopping");
 
         stopField.setAccessible(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 1413c7e..5371f56 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -72,10 +72,10 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemHandshakeSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoop20FileSystemLoopbackPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemLoopbackPrimarySelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopDualSyncSelfTest.class.getName())));
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoopDualAsyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualSyncSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfsDualAsyncSelfTest.class.getName())));
 
         suite.addTest(IgfsEventsTestSuite.suiteNoarchOnly());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
index 1f428e4..87233fc 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
@@ -54,7 +54,7 @@ public class IgniteIgfsLinuxAndMacOSTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(IgniteHadoopFileSystemIpcCacheSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(IgfsHadoop20FileSystemShmemPrimarySelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopIgfs20FileSystemShmemPrimarySelfTest.class.getName())));
 
         suite.addTest(IgfsEventsTestSuite.suite());
 


[18/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
deleted file mode 100644
index c4fc599..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
+++ /dev/null
@@ -1,87 +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.v2;
-
-import org.apache.hadoop.mapreduce.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-
-import java.io.*;
-
-/**
- * Adapter from own counter implementation into Hadoop API Counter od version 2.0.
- */
-public class GridHadoopV2Counter implements Counter {
-    /** Delegate. */
-    private final HadoopLongCounter cntr;
-
-    /**
-     * Creates new instance with given delegate.
-     *
-     * @param cntr Internal counter.
-     */
-    public GridHadoopV2Counter(HadoopLongCounter cntr) {
-        assert cntr != null : "counter must be non-null";
-
-        this.cntr = cntr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDisplayName(String displayName) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getName() {
-        return cntr.name();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getDisplayName() {
-        return getName();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getValue() {
-        return cntr.value();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setValue(long val) {
-        cntr.value(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void increment(long incr) {
-        cntr.increment(incr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Counter getUnderlyingCounter() {
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(DataOutput out) throws IOException {
-        throw new UnsupportedOperationException("not implemented");
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFields(DataInput in) throws IOException {
-        throw new UnsupportedOperationException("not implemented");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
deleted file mode 100644
index 5145bdb..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
+++ /dev/null
@@ -1,280 +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.v2;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapred.JobID;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.split.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
-import org.apache.ignite.internal.processors.hadoop.v1.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jdk8.backport.*;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.util.*;
-import java.util.Queue;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
-
-/**
- * Hadoop job implementation for v2 API.
- */
-public class GridHadoopV2Job implements GridHadoopJob {
-    /** */
-    private final JobConf jobConf;
-
-    /** */
-    private final JobContextImpl jobCtx;
-
-    /** Hadoop job ID. */
-    private final GridHadoopJobId jobId;
-
-    /** Job info. */
-    protected GridHadoopJobInfo jobInfo;
-
-    /** */
-    private final JobID hadoopJobID;
-
-    /** */
-    private final GridHadoopV2JobResourceManager rsrcMgr;
-
-    /** */
-    private final ConcurrentMap<T2<GridHadoopTaskType, Integer>, GridFutureAdapter<GridHadoopTaskContext>> ctxs =
-        new ConcurrentHashMap8<>();
-
-    /** Pooling task context class and thus class loading environment. */
-    private final Queue<Class<?>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
-
-    /** Local node ID */
-    private UUID locNodeId;
-
-    /** Serialized JobConf. */
-    private volatile byte[] jobConfData;
-
-    /**
-     * @param jobId Job ID.
-     * @param jobInfo Job info.
-     * @param log Logger.
-     */
-    public GridHadoopV2Job(GridHadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log) {
-        assert jobId != null;
-        assert jobInfo != null;
-
-        this.jobId = jobId;
-        this.jobInfo = jobInfo;
-
-        hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
-
-        HadoopClassLoader clsLdr = (HadoopClassLoader)getClass().getClassLoader();
-
-        // Before create JobConf instance we should set new context class loader.
-        Thread.currentThread().setContextClassLoader(clsLdr);
-
-        jobConf = new JobConf();
-
-        HadoopFileSystemsUtils.setupFileSystems(jobConf);
-
-        Thread.currentThread().setContextClassLoader(null);
-
-        for (Map.Entry<String,String> e : jobInfo.properties().entrySet())
-            jobConf.set(e.getKey(), e.getValue());
-
-        jobCtx = new JobContextImpl(jobConf, hadoopJobID);
-
-        rsrcMgr = new GridHadoopV2JobResourceManager(jobId, jobCtx, log);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobId id() {
-        return jobId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobInfo info() {
-        return jobInfo;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<GridHadoopInputSplit> input() throws IgniteCheckedException {
-        Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
-
-        try {
-            String jobDirPath = jobConf.get(MRJobConfig.MAPREDUCE_JOB_DIR);
-
-            if (jobDirPath == null) { // Probably job was submitted not by hadoop client.
-                // Assume that we have needed classes and try to generate input splits ourself.
-                if (jobConf.getUseNewMapper())
-                    return GridHadoopV2Splitter.splitJob(jobCtx);
-                else
-                    return GridHadoopV1Splitter.splitJob(jobConf);
-            }
-
-            Path jobDir = new Path(jobDirPath);
-
-            try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf)) {
-                JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
-                    jobDir);
-
-                if (F.isEmpty(metaInfos))
-                    throw new IgniteCheckedException("No input splits found.");
-
-                Path splitsFile = JobSubmissionFiles.getJobSplitFile(jobDir);
-
-                try (FSDataInputStream in = fs.open(splitsFile)) {
-                    Collection<GridHadoopInputSplit> res = new ArrayList<>(metaInfos.length);
-
-                    for (JobSplit.TaskSplitMetaInfo metaInfo : metaInfos) {
-                        long off = metaInfo.getStartOffset();
-
-                        String[] hosts = metaInfo.getLocations();
-
-                        in.seek(off);
-
-                        String clsName = Text.readString(in);
-
-                        GridHadoopFileBlock block = GridHadoopV1Splitter.readFileBlock(clsName, in, hosts);
-
-                        if (block == null)
-                            block = GridHadoopV2Splitter.readFileBlock(clsName, in, hosts);
-
-                        res.add(block != null ? block : new GridHadoopExternalSplit(hosts, off));
-                    }
-
-                    return res;
-                }
-            }
-            catch (Throwable e) {
-                throw transformException(e);
-            }
-        }
-        finally {
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException {
-        T2<GridHadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
-
-        GridFutureAdapter<GridHadoopTaskContext> fut = ctxs.get(locTaskId);
-
-        if (fut != null)
-            return fut.get();
-
-        GridFutureAdapter<GridHadoopTaskContext> old = ctxs.putIfAbsent(locTaskId, fut = new GridFutureAdapter<>());
-
-        if (old != null)
-            return old.get();
-
-        Class<?> cls = taskCtxClsPool.poll();
-
-        try {
-            if (cls == null) {
-                // If there is no pooled class, then load new one.
-                HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath());
-
-                cls = ldr.loadClass(GridHadoopV2TaskContext.class.getName());
-            }
-
-            Constructor<?> ctr = cls.getConstructor(GridHadoopTaskInfo.class, GridHadoopJob.class,
-                GridHadoopJobId.class, UUID.class, DataInput.class);
-
-            if (jobConfData == null)
-                synchronized(jobConf) {
-                    if (jobConfData == null) {
-                        ByteArrayOutputStream buf = new ByteArrayOutputStream();
-
-                        jobConf.write(new DataOutputStream(buf));
-
-                        jobConfData = buf.toByteArray();
-                    }
-                }
-
-            GridHadoopTaskContext res = (GridHadoopTaskContext)ctr.newInstance(info, this, jobId, locNodeId,
-                new DataInputStream(new ByteArrayInputStream(jobConfData)));
-
-            fut.onDone(res);
-
-            return res;
-        }
-        catch (Throwable e) {
-            IgniteCheckedException te = transformException(e);
-
-            fut.onDone(te);
-
-            throw te;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException {
-        this.locNodeId = locNodeId;
-
-        Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
-
-        try {
-            rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(locNodeId, jobId));
-        }
-        finally {
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void dispose(boolean external) throws IgniteCheckedException {
-        if (rsrcMgr != null && !external) {
-            File jobLocDir = jobLocalDir(locNodeId, jobId);
-
-            if (jobLocDir.exists())
-                U.delete(jobLocDir);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
-        rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
-        GridHadoopTaskContext ctx = ctxs.remove(new T2<>(info.type(), info.taskNumber())).get();
-
-        taskCtxClsPool.offer(ctx.getClass());
-
-        File locDir = taskLocalDir(locNodeId, info);
-
-        if (locDir.exists())
-            U.delete(locDir);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cleanupStagingDirectory() {
-        if (rsrcMgr != null)
-            rsrcMgr.cleanupStagingDirectory();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
deleted file mode 100644
index fdda3f6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
+++ /dev/null
@@ -1,305 +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.v2;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.file.*;
-import java.util.*;
-
-/**
- * Provides all resources are needed to the job execution. Downloads the main jar, the configuration and additional
- * files are needed to be placed on local files system.
- */
-public class GridHadoopV2JobResourceManager {
-    /** Hadoop job context. */
-    private final JobContextImpl ctx;
-
-    /** Logger. */
-    private final IgniteLogger log;
-
-    /** Job ID. */
-    private final GridHadoopJobId jobId;
-
-    /** Class path list. */
-    private URL[] clsPath;
-
-    /** Set of local resources. */
-    private final Collection<File> rsrcSet = new HashSet<>();
-
-    /** Staging directory to delivery job jar and config to the work nodes. */
-    private Path stagingDir;
-
-    /**
-     * Creates new instance.
-     * @param jobId Job ID.
-     * @param ctx Hadoop job context.
-     * @param log Logger.
-     */
-    public GridHadoopV2JobResourceManager(GridHadoopJobId jobId, JobContextImpl ctx, IgniteLogger log) {
-        this.jobId = jobId;
-        this.ctx = ctx;
-        this.log = log.getLogger(GridHadoopV2JobResourceManager.class);
-    }
-
-    /**
-     * Set working directory in local file system.
-     *
-     * @param dir Working directory.
-     * @throws IOException If fails.
-     */
-    private void setLocalFSWorkingDirectory(File dir) throws IOException {
-        JobConf cfg = ctx.getJobConf();
-
-        Thread.currentThread().setContextClassLoader(cfg.getClassLoader());
-
-        try {
-            cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
-
-            if(!cfg.getBoolean("fs.file.impl.disable.cache", false))
-                FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath()));
-        }
-        finally {
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /**
-     * Prepare job resources. Resolve the classpath list and download it if needed.
-     *
-     * @param download {@code true} If need to download resources.
-     * @param jobLocDir Work directory for the job.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void prepareJobEnvironment(boolean download, File jobLocDir) throws IgniteCheckedException {
-        try {
-            if (jobLocDir.exists())
-                throw new IgniteCheckedException("Local job directory already exists: " + jobLocDir.getAbsolutePath());
-
-            JobConf cfg = ctx.getJobConf();
-
-            String mrDir = cfg.get("mapreduce.job.dir");
-
-            if (mrDir != null) {
-                stagingDir = new Path(new URI(mrDir));
-
-                if (download) {
-                    FileSystem fs = FileSystem.get(stagingDir.toUri(), cfg);
-
-                    if (!fs.exists(stagingDir))
-                        throw new IgniteCheckedException("Failed to find map-reduce submission directory (does not exist): " +
-                            stagingDir);
-
-                    if (!FileUtil.copy(fs, stagingDir, jobLocDir, false, cfg))
-                        throw new IgniteCheckedException("Failed to copy job submission directory contents to local file system " +
-                            "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath() + ", jobId=" + jobId + ']');
-                }
-
-                File jarJobFile = new File(jobLocDir, "job.jar");
-
-                Collection<URL> clsPathUrls = new ArrayList<>();
-
-                clsPathUrls.add(jarJobFile.toURI().toURL());
-
-                rsrcSet.add(jarJobFile);
-                rsrcSet.add(new File(jobLocDir, "job.xml"));
-
-                processFiles(jobLocDir, ctx.getCacheFiles(), download, false, null, MRJobConfig.CACHE_LOCALFILES);
-                processFiles(jobLocDir, ctx.getCacheArchives(), download, true, null, MRJobConfig.CACHE_LOCALARCHIVES);
-                processFiles(jobLocDir, ctx.getFileClassPaths(), download, false, clsPathUrls, null);
-                processFiles(jobLocDir, ctx.getArchiveClassPaths(), download, true, clsPathUrls, null);
-
-                if (!clsPathUrls.isEmpty()) {
-                    clsPath = new URL[clsPathUrls.size()];
-
-                    clsPathUrls.toArray(clsPath);
-                }
-            }
-            else if (!jobLocDir.mkdirs())
-                throw new IgniteCheckedException("Failed to create local job directory: " + jobLocDir.getAbsolutePath());
-
-            setLocalFSWorkingDirectory(jobLocDir);
-        }
-        catch (URISyntaxException | IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /**
-     * Process list of resources.
-     *
-     * @param jobLocDir Job working directory.
-     * @param files Array of {@link java.net.URI} or {@link org.apache.hadoop.fs.Path} to process resources.
-     * @param download {@code true}, if need to download. Process class path only else.
-     * @param extract {@code true}, if need to extract archive.
-     * @param clsPathUrls Collection to add resource as classpath resource.
-     * @param rsrcNameProp Property for resource name array setting.
-     * @throws IOException If failed.
-     */
-    private void processFiles(File jobLocDir, @Nullable Object[] files, boolean download, boolean extract,
-        @Nullable Collection<URL> clsPathUrls, @Nullable String rsrcNameProp) throws IOException {
-        if (F.isEmptyOrNulls(files))
-            return;
-
-        Collection<String> res = new ArrayList<>();
-
-        for (Object pathObj : files) {
-            String locName = null;
-            Path srcPath;
-
-            if (pathObj instanceof URI) {
-                URI uri = (URI)pathObj;
-
-                locName = uri.getFragment();
-
-                srcPath = new Path(uri);
-            }
-            else
-                srcPath = (Path)pathObj;
-
-            if (locName == null)
-                locName = srcPath.getName();
-
-            File dstPath = new File(jobLocDir.getAbsolutePath(), locName);
-
-            res.add(locName);
-
-            rsrcSet.add(dstPath);
-
-            if (clsPathUrls != null)
-                clsPathUrls.add(dstPath.toURI().toURL());
-
-            if (!download)
-                continue;
-
-            JobConf cfg = ctx.getJobConf();
-
-            FileSystem dstFs = FileSystem.getLocal(cfg);
-
-            FileSystem srcFs = srcPath.getFileSystem(cfg);
-
-            if (extract) {
-                File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
-
-                if (!archivesPath.exists() && !archivesPath.mkdir())
-                    throw new IOException("Failed to create directory " +
-                         "[path=" + archivesPath + ", jobId=" + jobId + ']');
-
-                File archiveFile = new File(archivesPath, locName);
-
-                FileUtil.copy(srcFs, srcPath, dstFs, new Path(archiveFile.toString()), false, cfg);
-
-                String archiveNameLC = archiveFile.getName().toLowerCase();
-
-                if (archiveNameLC.endsWith(".jar"))
-                    RunJar.unJar(archiveFile, dstPath);
-                else if (archiveNameLC.endsWith(".zip"))
-                    FileUtil.unZip(archiveFile, dstPath);
-                else if (archiveNameLC.endsWith(".tar.gz") ||
-                    archiveNameLC.endsWith(".tgz") ||
-                    archiveNameLC.endsWith(".tar"))
-                    FileUtil.unTar(archiveFile, dstPath);
-                else
-                    throw new IOException("Cannot unpack archive [path=" + srcPath + ", jobId=" + jobId + ']');
-            }
-            else
-                FileUtil.copy(srcFs, srcPath, dstFs, new Path(dstPath.toString()), false, cfg);
-        }
-
-        if (!res.isEmpty() && rsrcNameProp != null)
-            ctx.getJobConf().setStrings(rsrcNameProp, res.toArray(new String[res.size()]));
-    }
-
-    /**
-     * Prepares working directory for the task.
-     *
-     * <ul>
-     *     <li>Creates working directory.</li>
-     *     <li>Creates symbolic links to all job resources in working directory.</li>
-     * </ul>
-     *
-     * @param path Path to working directory of the task.
-     * @throws IgniteCheckedException If fails.
-     */
-    public void prepareTaskWorkDir(File path) throws IgniteCheckedException {
-        try {
-            if (path.exists())
-                throw new IOException("Task local directory already exists: " + path);
-
-            if (!path.mkdir())
-                throw new IOException("Failed to create directory: " + path);
-
-            for (File resource : rsrcSet) {
-                File symLink = new File(path, resource.getName());
-
-                try {
-                    Files.createSymbolicLink(symLink.toPath(), resource.toPath());
-                }
-                catch (IOException e) {
-                    String msg = "Unable to create symlink \"" + symLink + "\" to \"" + resource + "\".";
-
-                    if (U.isWindows() && e instanceof FileSystemException)
-                        msg += "\n\nAbility to create symbolic links is required!\n" +
-                                "On Windows platform you have to grant permission 'Create symbolic links'\n" +
-                                "to your user or run the Accelerator as Administrator.\n";
-
-                    throw new IOException(msg, e);
-                }
-            }
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Unable to prepare local working directory for the task " +
-                 "[jobId=" + jobId + ", path=" + path+ ']', e);
-        }
-    }
-
-    /**
-     * Cleans up job staging directory.
-     */
-    public void cleanupStagingDirectory() {
-        try {
-            if (stagingDir != null)
-                stagingDir.getFileSystem(ctx.getJobConf()).delete(stagingDir, true);
-        }
-        catch (Exception e) {
-            log.error("Failed to remove job staging directory [path=" + stagingDir + ", jobId=" + jobId + ']' , e);
-        }
-    }
-
-    /**
-     * Returns array of class path for current job.
-     *
-     * @return Class path collection.
-     */
-    @Nullable public URL[] classPath() {
-        return clsPath;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2MapTask.java
deleted file mode 100644
index be0bea2..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2MapTask.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.hadoop.v2;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.map.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.*;
-
-/**
- * Hadoop map task implementation for v2 API.
- */
-public class GridHadoopV2MapTask extends GridHadoopV2Task {
-    /**
-     * @param taskInfo Task info.
-     */
-    public GridHadoopV2MapTask(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"ConstantConditions", "unchecked"})
-    @Override public void run0(GridHadoopV2TaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopInputSplit split = info().inputSplit();
-
-        InputSplit nativeSplit;
-
-        if (split instanceof GridHadoopFileBlock) {
-            GridHadoopFileBlock block = (GridHadoopFileBlock)split;
-
-            nativeSplit = new FileSplit(new Path(block.file().toString()), block.start(), block.length(), null);
-        }
-        else
-            nativeSplit = (InputSplit)taskCtx.getNativeSplit(split);
-
-        assert nativeSplit != null;
-
-        OutputFormat outputFormat = null;
-        Exception err = null;
-
-        JobContextImpl jobCtx = taskCtx.jobContext();
-
-        try {
-            InputFormat inFormat = ReflectionUtils.newInstance(jobCtx.getInputFormatClass(),
-                hadoopContext().getConfiguration());
-
-            RecordReader reader = inFormat.createRecordReader(nativeSplit, hadoopContext());
-
-            reader.initialize(nativeSplit, hadoopContext());
-
-            hadoopContext().reader(reader);
-
-            GridHadoopJobInfo jobInfo = taskCtx.job().info();
-
-            outputFormat = jobInfo.hasCombiner() || jobInfo.hasReducer() ? null : prepareWriter(jobCtx);
-
-            Mapper mapper = ReflectionUtils.newInstance(jobCtx.getMapperClass(), hadoopContext().getConfiguration());
-
-            try {
-                mapper.run(new WrappedMapper().getMapContext(hadoopContext()));
-            }
-            finally {
-                closeWriter();
-            }
-
-            commit(outputFormat);
-        }
-        catch (InterruptedException e) {
-            err = e;
-
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
-        catch (Exception e) {
-            err = e;
-
-            throw new IgniteCheckedException(e);
-        }
-        finally {
-            if (err != null)
-                abort(outputFormat);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Partitioner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Partitioner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Partitioner.java
deleted file mode 100644
index 0883520..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Partitioner.java
+++ /dev/null
@@ -1,44 +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.v2;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-/**
- * Hadoop partitioner adapter for v2 API.
- */
-public class GridHadoopV2Partitioner implements GridHadoopPartitioner {
-    /** Partitioner instance. */
-    private Partitioner<Object, Object> part;
-
-    /**
-     * @param cls Hadoop partitioner class.
-     * @param conf Job configuration.
-     */
-    public GridHadoopV2Partitioner(Class<? extends Partitioner<?, ?>> cls, Configuration conf) {
-        part = (Partitioner<Object, Object>) ReflectionUtils.newInstance(cls, conf);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int partition(Object key, Object val, int parts) {
-        return part.getPartition(key, val, parts);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2ReduceTask.java
deleted file mode 100644
index 146e05c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2ReduceTask.java
+++ /dev/null
@@ -1,88 +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.v2;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.reduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.*;
-
-/**
- * Hadoop reduce task implementation for v2 API.
- */
-public class GridHadoopV2ReduceTask extends GridHadoopV2Task {
-    /** {@code True} if reduce, {@code false} if combine. */
-    private final boolean reduce;
-
-    /**
-     * Constructor.
-     *
-     * @param taskInfo Task info.
-     * @param reduce {@code True} if reduce, {@code false} if combine.
-     */
-    public GridHadoopV2ReduceTask(GridHadoopTaskInfo taskInfo, boolean reduce) {
-        super(taskInfo);
-
-        this.reduce = reduce;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings({"ConstantConditions", "unchecked"})
-    @Override public void run0(GridHadoopV2TaskContext taskCtx) throws IgniteCheckedException {
-        OutputFormat outputFormat = null;
-        Exception err = null;
-
-        JobContextImpl jobCtx = taskCtx.jobContext();
-
-        try {
-            outputFormat = reduce || !taskCtx.job().info().hasReducer() ? prepareWriter(jobCtx) : null;
-
-            Reducer reducer = ReflectionUtils.newInstance(reduce ? jobCtx.getReducerClass() : jobCtx.getCombinerClass(),
-                jobCtx.getConfiguration());
-
-            try {
-                reducer.run(new WrappedReducer().getReducerContext(hadoopContext()));
-            }
-            finally {
-                closeWriter();
-            }
-
-            commit(outputFormat);
-        }
-        catch (InterruptedException e) {
-            err = e;
-
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
-        catch (Exception e) {
-            err = e;
-
-            throw new IgniteCheckedException(e);
-        }
-        finally {
-            if (err != null)
-                abort(outputFormat);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2SetupTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2SetupTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2SetupTask.java
deleted file mode 100644
index 54eda25..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2SetupTask.java
+++ /dev/null
@@ -1,66 +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.v2;
-
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.OutputFormat;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.*;
-
-import java.io.*;
-
-/**
- * Hadoop setup task (prepares job).
- */
-public class GridHadoopV2SetupTask extends GridHadoopV2Task {
-    /**
-     * Constructor.
-     *
-     * @param taskInfo task info.
-     */
-    public GridHadoopV2SetupTask(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("ConstantConditions")
-    @Override protected void run0(GridHadoopV2TaskContext taskCtx) throws IgniteCheckedException {
-        try {
-            JobContextImpl jobCtx = taskCtx.jobContext();
-
-            OutputFormat outputFormat = getOutputFormat(jobCtx);
-
-            outputFormat.checkOutputSpecs(jobCtx);
-
-            OutputCommitter committer = outputFormat.getOutputCommitter(hadoopContext());
-
-            if (committer != null)
-                committer.setupJob(jobCtx);
-        }
-        catch (ClassNotFoundException | IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
deleted file mode 100644
index 68338a6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Splitter.java
+++ /dev/null
@@ -1,105 +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.v2;
-
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.mapreduce.lib.input.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Hadoop API v2 splitter.
- */
-public class GridHadoopV2Splitter {
-    /** */
-    private static final String[] EMPTY_HOSTS = {};
-
-    /**
-     * @param ctx Job context.
-     * @return Collection of mapped splits.
-     * @throws IgniteCheckedException If mapping failed.
-     */
-    public static Collection<GridHadoopInputSplit> splitJob(JobContext ctx) throws IgniteCheckedException {
-        try {
-            InputFormat<?, ?> format = ReflectionUtils.newInstance(ctx.getInputFormatClass(), ctx.getConfiguration());
-
-            assert format != null;
-
-            List<InputSplit> splits = format.getSplits(ctx);
-
-            Collection<GridHadoopInputSplit> res = new ArrayList<>(splits.size());
-
-            int id = 0;
-
-            for (InputSplit nativeSplit : splits) {
-                if (nativeSplit instanceof FileSplit) {
-                    FileSplit s = (FileSplit)nativeSplit;
-
-                    res.add(new GridHadoopFileBlock(s.getLocations(), s.getPath().toUri(), s.getStart(), s.getLength()));
-                }
-                else
-                    res.add(HadoopUtils.wrapSplit(id, nativeSplit, nativeSplit.getLocations()));
-
-                id++;
-            }
-
-            return res;
-        }
-        catch (IOException | ClassNotFoundException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteInterruptedCheckedException(e);
-        }
-    }
-
-    /**
-     * @param clsName Input split class name.
-     * @param in Input stream.
-     * @param hosts Optional hosts.
-     * @return File block or {@code null} if it is not a {@link FileSplit} instance.
-     * @throws IgniteCheckedException If failed.
-     */
-    public static GridHadoopFileBlock readFileBlock(String clsName, DataInput in, @Nullable String[] hosts)
-        throws IgniteCheckedException {
-        if (!FileSplit.class.getName().equals(clsName))
-            return null;
-
-        FileSplit split = new FileSplit();
-
-        try {
-            split.readFields(in);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-
-        if (hosts == null)
-            hosts = EMPTY_HOSTS;
-
-        return new GridHadoopFileBlock(hosts, split.getPath().toUri(), split.getStart(), split.getLength());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Task.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Task.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Task.java
deleted file mode 100644
index 37697c6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Task.java
+++ /dev/null
@@ -1,181 +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.v2;
-
-import org.apache.hadoop.mapreduce.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Extended Hadoop v2 task.
- */
-public abstract class GridHadoopV2Task extends GridHadoopTask {
-    /** Hadoop context. */
-    private GridHadoopV2Context hadoopCtx;
-
-    /**
-     * Constructor.
-     *
-     * @param taskInfo Task info.
-     */
-    protected GridHadoopV2Task(GridHadoopTaskInfo taskInfo) {
-        super(taskInfo);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void run(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        GridHadoopV2TaskContext ctx = (GridHadoopV2TaskContext)taskCtx;
-
-        hadoopCtx = new GridHadoopV2Context(ctx);
-
-        run0(ctx);
-    }
-
-    /**
-     * Internal task routine.
-     *
-     * @param taskCtx Task context.
-     * @throws IgniteCheckedException
-     */
-    protected abstract void run0(GridHadoopV2TaskContext taskCtx) throws IgniteCheckedException;
-
-    /**
-     * @return hadoop context.
-     */
-    protected GridHadoopV2Context hadoopContext() {
-        return hadoopCtx;
-    }
-
-    /**
-     * Create and configure an OutputFormat instance.
-     *
-     * @param jobCtx Job context.
-     * @return Instance of OutputFormat is specified in job configuration.
-     * @throws ClassNotFoundException If specified class not found.
-     */
-    protected OutputFormat getOutputFormat(JobContext jobCtx) throws ClassNotFoundException {
-        return ReflectionUtils.newInstance(jobCtx.getOutputFormatClass(), hadoopContext().getConfiguration());
-    }
-
-    /**
-     * Put write into Hadoop context and return associated output format instance.
-     *
-     * @param jobCtx Job context.
-     * @return Output format.
-     * @throws IgniteCheckedException In case of Grid exception.
-     * @throws InterruptedException In case of interrupt.
-     */
-    protected OutputFormat prepareWriter(JobContext jobCtx)
-        throws IgniteCheckedException, InterruptedException {
-        try {
-            OutputFormat outputFormat = getOutputFormat(jobCtx);
-
-            assert outputFormat != null;
-
-            OutputCommitter outCommitter = outputFormat.getOutputCommitter(hadoopCtx);
-
-            if (outCommitter != null)
-                outCommitter.setupTask(hadoopCtx);
-
-            RecordWriter writer = outputFormat.getRecordWriter(hadoopCtx);
-
-            hadoopCtx.writer(writer);
-
-            return outputFormat;
-        }
-        catch (IOException | ClassNotFoundException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /**
-     * Closes writer.
-     *
-     * @throws Exception If fails and logger hasn't been specified.
-     */
-    protected void closeWriter() throws Exception {
-        RecordWriter writer = hadoopCtx.writer();
-
-        if (writer != null)
-            writer.close(hadoopCtx);
-    }
-
-    /**
-     * Setup task.
-     *
-     * @param outputFormat Output format.
-     * @throws IOException In case of IO exception.
-     * @throws InterruptedException In case of interrupt.
-     */
-    protected void setup(@Nullable OutputFormat outputFormat) throws IOException, InterruptedException {
-        if (hadoopCtx.writer() != null) {
-            assert outputFormat != null;
-
-            outputFormat.getOutputCommitter(hadoopCtx).setupTask(hadoopCtx);
-        }
-    }
-
-    /**
-     * Commit task.
-     *
-     * @param outputFormat Output format.
-     * @throws IgniteCheckedException In case of Grid exception.
-     * @throws IOException In case of IO exception.
-     * @throws InterruptedException In case of interrupt.
-     */
-    protected void commit(@Nullable OutputFormat outputFormat) throws IgniteCheckedException, IOException, InterruptedException {
-        if (hadoopCtx.writer() != null) {
-            assert outputFormat != null;
-
-            OutputCommitter outputCommitter = outputFormat.getOutputCommitter(hadoopCtx);
-
-            if (outputCommitter.needsTaskCommit(hadoopCtx))
-                outputCommitter.commitTask(hadoopCtx);
-        }
-    }
-
-    /**
-     * Abort task.
-     *
-     * @param outputFormat Output format.
-     */
-    protected void abort(@Nullable OutputFormat outputFormat) {
-        if (hadoopCtx.writer() != null) {
-            assert outputFormat != null;
-
-            try {
-                outputFormat.getOutputCommitter(hadoopCtx).abortTask(hadoopCtx);
-            }
-            catch (IOException ignore) {
-                // Ignore.
-            }
-            catch (InterruptedException ignore) {
-                Thread.currentThread().interrupt();
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cancel() {
-        hadoopCtx.cancel();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
deleted file mode 100644
index 476d8f0..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
+++ /dev/null
@@ -1,443 +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.v2;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.io.*;
-import org.apache.hadoop.io.serializer.*;
-import org.apache.hadoop.mapred.*;
-import org.apache.hadoop.mapred.JobID;
-import org.apache.hadoop.mapred.TaskAttemptID;
-import org.apache.hadoop.mapred.TaskID;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
-import org.apache.ignite.internal.processors.hadoop.v1.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
-import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
-
-/**
- * Context for task execution.
- */
-public class GridHadoopV2TaskContext extends GridHadoopTaskContext {
-    /** */
-    private static final boolean COMBINE_KEY_GROUPING_SUPPORTED;
-
-    /**
-     * Check for combiner grouping support (available since Hadoop 2.3).
-     */
-    static {
-        boolean ok;
-
-        try {
-            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
-
-            ok = true;
-        }
-        catch (NoSuchMethodException ignore) {
-            ok = false;
-        }
-
-        COMBINE_KEY_GROUPING_SUPPORTED = ok;
-    }
-
-    /** Flag is set if new context-object code is used for running the mapper. */
-    private final boolean useNewMapper;
-
-    /** Flag is set if new context-object code is used for running the reducer. */
-    private final boolean useNewReducer;
-
-    /** Flag is set if new context-object code is used for running the combiner. */
-    private final boolean useNewCombiner;
-
-    /** */
-    private final JobContextImpl jobCtx;
-
-    /** Set if task is to cancelling. */
-    private volatile boolean cancelled;
-
-    /** Current task. */
-    private volatile GridHadoopTask task;
-
-    /** Local node ID */
-    private UUID locNodeId;
-
-    /** Counters for task. */
-    private final GridHadoopCounters cntrs = new HadoopCountersImpl();
-
-    /**
-     * @param taskInfo Task info.
-     * @param job Job.
-     * @param jobId Job ID.
-     * @param locNodeId Local node ID.
-     * @param jobConfDataInput DataInput for read JobConf.
-     */
-    public GridHadoopV2TaskContext(GridHadoopTaskInfo taskInfo, GridHadoopJob job, GridHadoopJobId jobId,
-        @Nullable UUID locNodeId, DataInput jobConfDataInput) throws IgniteCheckedException {
-        super(taskInfo, job);
-        this.locNodeId = locNodeId;
-
-        // Before create JobConf instance we should set new context class loader.
-        Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
-
-        try {
-            JobConf jobConf = new JobConf();
-
-            try {
-                jobConf.readFields(jobConfDataInput);
-            }
-            catch (IOException e) {
-                throw new IgniteCheckedException(e);
-            }
-
-            // For map-reduce jobs prefer local writes.
-            jobConf.setBooleanIfUnset(PARAM_IGFS_PREFER_LOCAL_WRITES, true);
-
-            jobCtx = new JobContextImpl(jobConf, new JobID(jobId.globalId().toString(), jobId.localId()));
-
-            useNewMapper = jobConf.getUseNewMapper();
-            useNewReducer = jobConf.getUseNewReducer();
-            useNewCombiner = jobConf.getCombinerClass() == null;
-        }
-        finally {
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends GridHadoopCounter> T counter(String grp, String name, Class<T> cls) {
-        return cntrs.counter(grp, name, cls);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopCounters counters() {
-        return cntrs;
-    }
-
-    /**
-     * Creates appropriate task from current task info.
-     *
-     * @return Task.
-     */
-    private GridHadoopTask createTask() {
-        boolean isAbort = taskInfo().type() == GridHadoopTaskType.ABORT;
-
-        switch (taskInfo().type()) {
-            case SETUP:
-                return useNewMapper ? new GridHadoopV2SetupTask(taskInfo()) : new GridHadoopV1SetupTask(taskInfo());
-
-            case MAP:
-                return useNewMapper ? new GridHadoopV2MapTask(taskInfo()) : new GridHadoopV1MapTask(taskInfo());
-
-            case REDUCE:
-                return useNewReducer ? new GridHadoopV2ReduceTask(taskInfo(), true) :
-                    new GridHadoopV1ReduceTask(taskInfo(), true);
-
-            case COMBINE:
-                return useNewCombiner ? new GridHadoopV2ReduceTask(taskInfo(), false) :
-                    new GridHadoopV1ReduceTask(taskInfo(), false);
-
-            case COMMIT:
-            case ABORT:
-                return useNewReducer ? new GridHadoopV2CleanupTask(taskInfo(), isAbort) :
-                    new GridHadoopV1CleanupTask(taskInfo(), isAbort);
-
-            default:
-                return null;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void run() throws IgniteCheckedException {
-        try {
-            Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
-
-            try {
-                task = createTask();
-            }
-            catch (Throwable e) {
-                throw transformException(e);
-            }
-
-            if (cancelled)
-                throw new HadoopTaskCancelledException("Task cancelled.");
-
-            try {
-                task.run(this);
-            }
-            catch (Throwable e) {
-                throw transformException(e);
-            }
-        }
-        finally {
-            task = null;
-
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cancel() {
-        cancelled = true;
-
-        GridHadoopTask t = task;
-
-        if (t != null)
-            t.cancel();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareTaskEnvironment() throws IgniteCheckedException {
-        File locDir;
-
-        switch(taskInfo().type()) {
-            case MAP:
-            case REDUCE:
-                job().prepareTaskEnvironment(taskInfo());
-
-                locDir = taskLocalDir(locNodeId, taskInfo());
-
-                break;
-
-            default:
-                locDir = jobLocalDir(locNodeId, taskInfo().jobId());
-        }
-
-        Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
-
-        try {
-            FileSystem fs = FileSystem.get(jobConf());
-
-            HadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
-
-            LocalFileSystem locFs = FileSystem.getLocal(jobConf());
-
-            locFs.setWorkingDirectory(new Path(locDir.getAbsolutePath()));
-        }
-        catch (Throwable e) {
-            throw transformException(e);
-        }
-        finally {
-            Thread.currentThread().setContextClassLoader(null);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cleanupTaskEnvironment() throws IgniteCheckedException {
-        job().cleanupTaskEnvironment(taskInfo());
-    }
-
-    /**
-     * Creates Hadoop attempt ID.
-     *
-     * @return Attempt ID.
-     */
-    public TaskAttemptID attemptId() {
-        TaskID tid = new TaskID(jobCtx.getJobID(), taskType(taskInfo().type()), taskInfo().taskNumber());
-
-        return new TaskAttemptID(tid, taskInfo().attempt());
-    }
-
-    /**
-     * @param type Task type.
-     * @return Hadoop task type.
-     */
-    private TaskType taskType(GridHadoopTaskType type) {
-        switch (type) {
-            case SETUP:
-                return TaskType.JOB_SETUP;
-            case MAP:
-            case COMBINE:
-                return TaskType.MAP;
-
-            case REDUCE:
-                return TaskType.REDUCE;
-
-            case COMMIT:
-            case ABORT:
-                return TaskType.JOB_CLEANUP;
-
-            default:
-                return null;
-        }
-    }
-
-    /**
-     * Gets job configuration of the task.
-     *
-     * @return Job configuration.
-     */
-    public JobConf jobConf() {
-        return jobCtx.getJobConf();
-    }
-
-    /**
-     * Gets job context of the task.
-     *
-     * @return Job context.
-     */
-    public JobContextImpl jobContext() {
-        return jobCtx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopPartitioner partitioner() throws IgniteCheckedException {
-        Class<?> partClsOld = jobConf().getClass("mapred.partitioner.class", null);
-
-        if (partClsOld != null)
-            return new GridHadoopV1Partitioner(jobConf().getPartitionerClass(), jobConf());
-
-        try {
-            return new GridHadoopV2Partitioner(jobCtx.getPartitionerClass(), jobConf());
-        }
-        catch (ClassNotFoundException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /**
-     * Gets serializer for specified class.
-     *
-     * @param cls Class.
-     * @param jobConf Job configuration.
-     * @return Appropriate serializer.
-     */
-    @SuppressWarnings("unchecked")
-    private GridHadoopSerialization getSerialization(Class<?> cls, Configuration jobConf) throws IgniteCheckedException {
-        A.notNull(cls, "cls");
-
-        SerializationFactory factory = new SerializationFactory(jobConf);
-
-        Serialization<?> serialization = factory.getSerialization(cls);
-
-        if (serialization == null)
-            throw new IgniteCheckedException("Failed to find serialization for: " + cls.getName());
-
-        if (serialization.getClass() == WritableSerialization.class)
-            return new GridHadoopWritableSerialization((Class<? extends Writable>)cls);
-
-        return new GridHadoopSerializationWrapper(serialization, cls);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopSerialization keySerialization() throws IgniteCheckedException {
-        return getSerialization(jobCtx.getMapOutputKeyClass(), jobConf());
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopSerialization valueSerialization() throws IgniteCheckedException {
-        return getSerialization(jobCtx.getMapOutputValueClass(), jobConf());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Comparator<Object> sortComparator() {
-        return (Comparator<Object>)jobCtx.getSortComparator();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Comparator<Object> groupComparator() {
-        Comparator<?> res;
-
-        switch (taskInfo().type()) {
-            case COMBINE:
-                res = COMBINE_KEY_GROUPING_SUPPORTED ?
-                    jobContext().getCombinerKeyGroupingComparator() : jobContext().getGroupingComparator();
-
-                break;
-
-            case REDUCE:
-                res = jobContext().getGroupingComparator();
-
-                break;
-
-            default:
-                return null;
-        }
-
-        if (res != null && res.getClass() != sortComparator().getClass())
-            return (Comparator<Object>)res;
-
-        return null;
-    }
-
-    /**
-     * @param split Split.
-     * @return Native Hadoop split.
-     * @throws IgniteCheckedException if failed.
-     */
-    @SuppressWarnings("unchecked")
-    public Object getNativeSplit(GridHadoopInputSplit split) throws IgniteCheckedException {
-        if (split instanceof GridHadoopExternalSplit)
-            return readExternalSplit((GridHadoopExternalSplit)split);
-
-        if (split instanceof GridHadoopSplitWrapper)
-            return unwrapSplit((GridHadoopSplitWrapper)split);
-
-        throw new IllegalStateException("Unknown split: " + split);
-    }
-
-    /**
-     * @param split External split.
-     * @return Native input split.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    private Object readExternalSplit(GridHadoopExternalSplit split) throws IgniteCheckedException {
-        Path jobDir = new Path(jobConf().get(MRJobConfig.MAPREDUCE_JOB_DIR));
-
-        try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf());
-            FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
-
-            in.seek(split.offset());
-
-            String clsName = Text.readString(in);
-
-            Class<?> cls = jobConf().getClassByName(clsName);
-
-            assert cls != null;
-
-            Serialization serialization = new SerializationFactory(jobConf()).getSerialization(cls);
-
-            Deserializer deserializer = serialization.getDeserializer(cls);
-
-            deserializer.open(in);
-
-            Object res = deserializer.deserialize(null);
-
-            deserializer.close();
-
-            assert res != null;
-
-            return res;
-        }
-        catch (IOException | ClassNotFoundException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopWritableSerialization.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopWritableSerialization.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopWritableSerialization.java
deleted file mode 100644
index 4361ad4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopWritableSerialization.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.v2;
-
-import org.apache.hadoop.io.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Optimized serialization for Hadoop {@link Writable} types.
- */
-public class GridHadoopWritableSerialization implements GridHadoopSerialization {
-    /** */
-    private final Class<? extends Writable> cls;
-
-    /**
-     * @param cls Class.
-     */
-    public GridHadoopWritableSerialization(Class<? extends Writable> cls) {
-        assert cls != null;
-
-        this.cls = cls;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(DataOutput out, Object obj) throws IgniteCheckedException {
-        assert cls.isAssignableFrom(obj.getClass()) : cls + " " + obj.getClass();
-
-        try {
-            ((Writable)obj).write(out);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object read(DataInput in, @Nullable Object obj) throws IgniteCheckedException {
-        Writable w = obj == null ? U.newInstance(cls) : cls.cast(obj);
-
-        try {
-            w.readFields(in);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-
-        return w;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java
new file mode 100644
index 0000000..8e968b2
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopExternalSplit.java
@@ -0,0 +1,87 @@
+/*
+ * 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.v2;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+
+import java.io.*;
+
+/**
+ * Split serialized in external file.
+ */
+public class HadoopExternalSplit extends GridHadoopInputSplit {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long off;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public HadoopExternalSplit() {
+        // No-op.
+    }
+
+    /**
+     * @param hosts Hosts.
+     * @param off Offset of this split in external file.
+     */
+    public HadoopExternalSplit(String[] hosts, long off) {
+        assert off >= 0 : off;
+        assert hosts != null;
+
+        this.hosts = hosts;
+        this.off = off;
+    }
+
+    /**
+     * @return Offset of this input split in external file.
+     */
+    public long offset() {
+        return off;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeLong(off);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        off = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        HadoopExternalSplit that = (HadoopExternalSplit) o;
+
+        return off == that.off;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return (int)(off ^ (off >>> 32));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopNativeCodeLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopNativeCodeLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopNativeCodeLoader.java
new file mode 100644
index 0000000..081182d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopNativeCodeLoader.java
@@ -0,0 +1,74 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.classification.*;
+import org.apache.hadoop.conf.*;
+
+/**
+ * A fake helper to load the native hadoop code i.e. libhadoop.so.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HadoopNativeCodeLoader {
+    /**
+     * Check if native-hadoop code is loaded for this platform.
+     *
+     * @return <code>true</code> if native-hadoop is loaded,
+     *         else <code>false</code>
+     */
+    public static boolean isNativeCodeLoaded() {
+        return false;
+    }
+
+    /**
+     * Returns true only if this build was compiled with support for snappy.
+     */
+    public static boolean buildSupportsSnappy() {
+        return false;
+    }
+
+    /**
+     * @return Library name.
+     */
+    public static String getLibraryName() {
+        throw new IllegalStateException();
+    }
+
+    /**
+     * Return if native hadoop libraries, if present, can be used for this job.
+     * @param conf configuration
+     *
+     * @return <code>true</code> if native hadoop libraries, if present, can be
+     *         used for this job; <code>false</code> otherwise.
+     */
+    public boolean getLoadNativeLibraries(Configuration conf) {
+        return false;
+    }
+
+    /**
+     * Set if native hadoop libraries, if present, can be used for this job.
+     *
+     * @param conf configuration
+     * @param loadNativeLibraries can native hadoop libraries be loaded
+     */
+    public void setLoadNativeLibraries(Configuration conf, boolean loadNativeLibraries) {
+        // No-op.
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java
new file mode 100644
index 0000000..3e4a4c4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSerializationWrapper.java
@@ -0,0 +1,133 @@
+/*
+ * 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.v2;
+
+import org.apache.hadoop.io.serializer.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * The wrapper around external serializer.
+ */
+public class HadoopSerializationWrapper<T> implements GridHadoopSerialization {
+    /** External serializer - writer. */
+    private final Serializer<T> serializer;
+
+    /** External serializer - reader. */
+    private final Deserializer<T> deserializer;
+
+    /** Data output for current write operation. */
+    private OutputStream currOut;
+
+    /** Data input for current read operation. */
+    private InputStream currIn;
+
+    /** Wrapper around current output to provide OutputStream interface. */
+    private final OutputStream outStream = new OutputStream() {
+        /** {@inheritDoc} */
+        @Override public void write(int b) throws IOException {
+            currOut.write(b);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(byte[] b, int off, int len) throws IOException {
+            currOut.write(b, off, len);
+        }
+    };
+
+    /** Wrapper around current input to provide InputStream interface. */
+    private final InputStream inStream = new InputStream() {
+        /** {@inheritDoc} */
+        @Override public int read() throws IOException {
+            return currIn.read();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int read(byte[] b, int off, int len) throws IOException {
+            return currIn.read(b, off, len);
+        }
+    };
+
+    /**
+     * @param serialization External serializer to wrap.
+     * @param cls The class to serialize.
+     */
+    public HadoopSerializationWrapper(Serialization<T> serialization, Class<T> cls) throws IgniteCheckedException {
+        assert cls != null;
+
+        serializer = serialization.getSerializer(cls);
+        deserializer = serialization.getDeserializer(cls);
+
+        try {
+            serializer.open(outStream);
+            deserializer.open(inStream);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(DataOutput out, Object obj) throws IgniteCheckedException {
+        assert out != null;
+        assert obj != null;
+
+        try {
+            currOut = (OutputStream)out;
+
+            serializer.serialize((T)obj);
+
+            currOut = null;
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object read(DataInput in, @Nullable Object obj) throws IgniteCheckedException {
+        assert in != null;
+
+        try {
+            currIn = (InputStream)in;
+
+            T res = deserializer.deserialize((T) obj);
+
+            currIn = null;
+
+            return res;
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IgniteCheckedException {
+        try {
+            serializer.close();
+            deserializer.close();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java
new file mode 100644
index 0000000..454f90a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopShutdownHookManager.java
@@ -0,0 +1,96 @@
+/*
+ * 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.v2;
+
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Fake manager for shutdown hooks.
+ */
+public class HadoopShutdownHookManager {
+    /** */
+    private static final HadoopShutdownHookManager MGR = new HadoopShutdownHookManager();
+
+    /**
+     * Return <code>ShutdownHookManager</code> singleton.
+     *
+     * @return <code>ShutdownHookManager</code> singleton.
+     */
+    public static HadoopShutdownHookManager get() {
+        return MGR;
+    }
+
+    /** */
+    private Set<Runnable> hooks = Collections.synchronizedSet(new HashSet<Runnable>());
+
+    /** */
+    private AtomicBoolean shutdownInProgress = new AtomicBoolean(false);
+
+    /**
+     * Singleton.
+     */
+    private HadoopShutdownHookManager() {
+        // No-op.
+    }
+
+    /**
+     * Adds a shutdownHook with a priority, the higher the priority
+     * the earlier will run. ShutdownHooks with same priority run
+     * in a non-deterministic order.
+     *
+     * @param shutdownHook shutdownHook <code>Runnable</code>
+     * @param priority priority of the shutdownHook.
+     */
+    public void addShutdownHook(Runnable shutdownHook, int priority) {
+        if (shutdownHook == null)
+            throw new IllegalArgumentException("shutdownHook cannot be NULL");
+
+        hooks.add(shutdownHook);
+    }
+
+    /**
+     * Removes a shutdownHook.
+     *
+     * @param shutdownHook shutdownHook to remove.
+     * @return TRUE if the shutdownHook was registered and removed,
+     * FALSE otherwise.
+     */
+    public boolean removeShutdownHook(Runnable shutdownHook) {
+        return hooks.remove(shutdownHook);
+    }
+
+    /**
+     * Indicates if a shutdownHook is registered or not.
+     *
+     * @param shutdownHook shutdownHook to check if registered.
+     * @return TRUE/FALSE depending if the shutdownHook is is registered.
+     */
+    public boolean hasShutdownHook(Runnable shutdownHook) {
+        return hooks.contains(shutdownHook);
+    }
+
+    /**
+     * Indicates if shutdown is in progress or not.
+     *
+     * @return TRUE if the shutdown is in progress, otherwise FALSE.
+     */
+    public boolean isShutdownInProgress() {
+        return shutdownInProgress.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java
new file mode 100644
index 0000000..c73a8b0
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopSplitWrapper.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.v2;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * The wrapper for native hadoop input splits.
+ *
+ * Warning!! This class must not depend on any Hadoop classes directly or indirectly.
+ */
+public class HadoopSplitWrapper extends GridHadoopInputSplit {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Native hadoop input split. */
+    private byte[] bytes;
+
+    /** */
+    private String clsName;
+
+    /** Internal ID */
+    private int id;
+
+    /**
+     * Creates new split wrapper.
+     */
+    public HadoopSplitWrapper() {
+        // No-op.
+    }
+
+    /**
+     * Creates new split wrapper.
+     *
+     * @param id Split ID.
+     * @param clsName Class name.
+     * @param bytes Serialized class.
+     * @param hosts Hosts where split is located.
+     */
+    public HadoopSplitWrapper(int id, String clsName, byte[] bytes, String[] hosts) {
+        assert hosts != null;
+        assert clsName != null;
+        assert bytes != null;
+
+        this.hosts = hosts;
+        this.id = id;
+
+        this.clsName = clsName;
+        this.bytes = bytes;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(id);
+
+        out.writeUTF(clsName);
+        U.writeByteArray(out, bytes);
+    }
+
+    /**
+     * @return Class name.
+     */
+    public String className() {
+        return clsName;
+    }
+
+    /**
+     * @return Class bytes.
+     */
+    public byte[] bytes() {
+        return bytes;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        id = in.readInt();
+
+        clsName = in.readUTF();
+        bytes = U.readByteArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        HadoopSplitWrapper that = (HadoopSplitWrapper)o;
+
+        return id == that.id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return id;
+    }
+}


[21/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java
new file mode 100644
index 0000000..3a94d43
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopExternalProcessStarter.java
@@ -0,0 +1,296 @@
+/*
+ * 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.taskexecutor.external.child;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.logger.log4j.*;
+import org.apache.ignite.marshaller.optimized.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Hadoop external process base class.
+ */
+public class HadoopExternalProcessStarter {
+    /** Path to Log4j configuration file. */
+    public static final String DFLT_LOG4J_CONFIG = "config/ignite-log4j.xml";
+
+    /** Arguments. */
+    private Args args;
+
+    /** System out. */
+    private OutputStream out;
+
+    /** System err. */
+    private OutputStream err;
+
+    /**
+     * @param args Parsed arguments.
+     */
+    public HadoopExternalProcessStarter(Args args) {
+        this.args = args;
+    }
+
+    /**
+     * @param cmdArgs Process arguments.
+     */
+    public static void main(String[] cmdArgs) {
+        try {
+            Args args = arguments(cmdArgs);
+
+            new HadoopExternalProcessStarter(args).run();
+        }
+        catch (Exception e) {
+            System.err.println("Failed");
+
+            System.err.println(e.getMessage());
+
+            e.printStackTrace(System.err);
+        }
+    }
+
+    /**
+     *
+     * @throws Exception
+     */
+    public void run() throws Exception {
+        U.setWorkDirectory(args.workDir, U.getIgniteHome());
+
+        File outputDir = outputDirectory();
+
+        initializeStreams(outputDir);
+
+        ExecutorService msgExecSvc = Executors.newFixedThreadPool(
+            Integer.getInteger("MSG_THREAD_POOL_SIZE", Runtime.getRuntime().availableProcessors() * 2));
+
+        IgniteLogger log = logger(outputDir);
+
+        HadoopExternalCommunication comm = new HadoopExternalCommunication(
+            args.nodeId,
+            args.childProcId,
+            new OptimizedMarshaller(),
+            log,
+            msgExecSvc,
+            "external"
+        );
+
+        comm.start();
+
+        HadoopProcessDescriptor nodeDesc = new HadoopProcessDescriptor(args.nodeId, args.parentProcId);
+        nodeDesc.address(args.addr);
+        nodeDesc.tcpPort(args.tcpPort);
+        nodeDesc.sharedMemoryPort(args.shmemPort);
+
+        HadoopChildProcessRunner runner = new HadoopChildProcessRunner();
+
+        runner.start(comm, nodeDesc, msgExecSvc, log);
+
+        System.err.println("Started");
+        System.err.flush();
+
+        System.setOut(new PrintStream(out));
+        System.setErr(new PrintStream(err));
+    }
+
+    /**
+     * @param outputDir Directory for process output.
+     * @throws Exception
+     */
+    private void initializeStreams(File outputDir) throws Exception {
+        out = new FileOutputStream(new File(outputDir, args.childProcId + ".out"));
+        err = new FileOutputStream(new File(outputDir, args.childProcId + ".err"));
+    }
+
+    /**
+     * @return Path to output directory.
+     * @throws IOException If failed.
+     */
+    private File outputDirectory() throws IOException {
+        File f = new File(args.out);
+
+        if (!f.exists()) {
+            if (!f.mkdirs())
+                throw new IOException("Failed to create output directory: " + args.out);
+        }
+        else {
+            if (f.isFile())
+                throw new IOException("Output directory is a file: " + args.out);
+        }
+
+        return f;
+    }
+
+    /**
+     * @param outputDir Directory for process output.
+     * @return Logger.
+     */
+    private IgniteLogger logger(final File outputDir) {
+        final URL url = U.resolveIgniteUrl(DFLT_LOG4J_CONFIG);
+
+        Log4JLogger logger;
+
+        try {
+            logger = url != null ? new Log4JLogger(url) : new Log4JLogger(true);
+        }
+        catch (IgniteCheckedException e) {
+            System.err.println("Failed to create URL-based logger. Will use default one.");
+
+            e.printStackTrace();
+
+            logger = new Log4JLogger(true);
+        }
+
+        logger.updateFilePath(new IgniteClosure<String, String>() {
+            @Override public String apply(String s) {
+                return new File(outputDir, args.childProcId + ".log").getAbsolutePath();
+            }
+        });
+
+        return logger;
+    }
+
+    /**
+     * @param processArgs Process arguments.
+     * @return Child process instance.
+     */
+    private static Args arguments(String[] processArgs) throws Exception {
+        Args args = new Args();
+
+        for (int i = 0; i < processArgs.length; i++) {
+            String arg = processArgs[i];
+
+            switch (arg) {
+                case "-cpid": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing process ID for '-cpid' parameter");
+
+                    String procIdStr = processArgs[++i];
+
+                    args.childProcId = UUID.fromString(procIdStr);
+
+                    break;
+                }
+
+                case "-ppid": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing process ID for '-ppid' parameter");
+
+                    String procIdStr = processArgs[++i];
+
+                    args.parentProcId = UUID.fromString(procIdStr);
+
+                    break;
+                }
+
+                case "-nid": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing node ID for '-nid' parameter");
+
+                    String nodeIdStr = processArgs[++i];
+
+                    args.nodeId = UUID.fromString(nodeIdStr);
+
+                    break;
+                }
+
+                case "-addr": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing node address for '-addr' parameter");
+
+                    args.addr = processArgs[++i];
+
+                    break;
+                }
+
+                case "-tport": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing tcp port for '-tport' parameter");
+
+                    args.tcpPort = Integer.parseInt(processArgs[++i]);
+
+                    break;
+                }
+
+                case "-sport": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing shared memory port for '-sport' parameter");
+
+                    args.shmemPort = Integer.parseInt(processArgs[++i]);
+
+                    break;
+                }
+
+                case "-out": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing output folder name for '-out' parameter");
+
+                    args.out = processArgs[++i];
+
+                    break;
+                }
+
+                case "-wd": {
+                    if (i == processArgs.length - 1)
+                        throw new Exception("Missing work folder name for '-wd' parameter");
+
+                    args.workDir = processArgs[++i];
+
+                    break;
+                }
+            }
+        }
+
+        return args;
+    }
+
+    /**
+     * Execution arguments.
+     */
+    private static class Args {
+        /** Process ID. */
+        private UUID childProcId;
+
+        /** Process ID. */
+        private UUID parentProcId;
+
+        /** Process ID. */
+        private UUID nodeId;
+
+        /** Node address. */
+        private String addr;
+
+        /** TCP port */
+        private int tcpPort;
+
+        /** Shmem port. */
+        private int shmemPort = -1;
+
+        /** Output folder. */
+        private String out;
+
+        /** Work directory. */
+        private String workDir;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopAbstractCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopAbstractCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopAbstractCommunicationClient.java
deleted file mode 100644
index 5dee79b..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopAbstractCommunicationClient.java
+++ /dev/null
@@ -1,96 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.concurrent.atomic.*;
-
-/**
- * Implements basic lifecycle for communication clients.
- */
-public abstract class GridHadoopAbstractCommunicationClient implements GridHadoopCommunicationClient {
-    /** Time when this client was last used. */
-    private volatile long lastUsed = U.currentTimeMillis();
-
-    /** Reservations. */
-    private final AtomicInteger reserves = new AtomicInteger();
-
-    /** {@inheritDoc} */
-    @Override public boolean close() {
-        return reserves.compareAndSet(0, -1);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void forceClose() {
-        reserves.set(-1);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean closed() {
-        return reserves.get() == -1;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean reserve() {
-        while (true) {
-            int r = reserves.get();
-
-            if (r == -1)
-                return false;
-
-            if (reserves.compareAndSet(r, r + 1))
-                return true;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void release() {
-        while (true) {
-            int r = reserves.get();
-
-            if (r == -1)
-                return;
-
-            if (reserves.compareAndSet(r, r - 1))
-                return;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean reserved() {
-        return reserves.get() > 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getIdleTime() {
-        return U.currentTimeMillis() - lastUsed;
-    }
-
-    /**
-     * Updates used time.
-     */
-    protected void markUsed() {
-        lastUsed = U.currentTimeMillis();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopAbstractCommunicationClient.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
deleted file mode 100644
index f4eb41a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
+++ /dev/null
@@ -1,72 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-
-/**
- *
- */
-public interface GridHadoopCommunicationClient {
-    /**
-     * @return {@code True} if client has been closed by this call,
-     *      {@code false} if failed to close client (due to concurrent reservation or concurrent close).
-     */
-    public boolean close();
-
-    /**
-     * Forces client close.
-     */
-    public void forceClose();
-
-    /**
-     * @return {@code True} if client is closed;
-     */
-    public boolean closed();
-
-    /**
-     * @return {@code True} if client was reserved, {@code false} otherwise.
-     */
-    public boolean reserve();
-
-    /**
-     * Releases this client by decreasing reservations.
-     */
-    public void release();
-
-    /**
-     * @return {@code True} if client was reserved.
-     */
-    public boolean reserved();
-
-    /**
-     * Gets idle time of this client.
-     *
-     * @return Idle time of this client.
-     */
-    public long getIdleTime();
-
-    /**
-     * @param desc Process descriptor.
-     * @param msg Message to send.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
deleted file mode 100644
index 937e245..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
+++ /dev/null
@@ -1,1431 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.ipc.*;
-import org.apache.ignite.internal.util.ipc.shmem.*;
-import org.apache.ignite.internal.util.nio.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.internal.util.worker.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.thread.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.*;
-import java.nio.channels.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Hadoop external communication class.
- */
-public class GridHadoopExternalCommunication {
-    /** IPC error message. */
-    public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
-        "(switching to TCP, may be slower)."; // TODO IGNITE-70 Add link to documentation
-
-    /** Default port which node sets listener to (value is <tt>47100</tt>). */
-    public static final int DFLT_PORT = 27100;
-
-    /** Default connection timeout (value is <tt>1000</tt>ms). */
-    public static final long DFLT_CONN_TIMEOUT = 1000;
-
-    /** Default Maximum connection timeout (value is <tt>600,000</tt>ms). */
-    public static final long DFLT_MAX_CONN_TIMEOUT = 10 * 60 * 1000;
-
-    /** Default reconnect attempts count (value is <tt>10</tt>). */
-    public static final int DFLT_RECONNECT_CNT = 10;
-
-    /** Default message queue limit per connection (for incoming and outgoing . */
-    public static final int DFLT_MSG_QUEUE_LIMIT = GridNioServer.DFLT_SEND_QUEUE_LIMIT;
-
-    /**
-     * Default count of selectors for TCP server equals to
-     * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
-     */
-    public static final int DFLT_SELECTORS_CNT = 1;
-
-    /** Node ID meta for session. */
-    private static final int PROCESS_META = GridNioSessionMetaKey.nextUniqueKey();
-
-    /** Handshake timeout meta for session. */
-    private static final int HANDSHAKE_FINISH_META = GridNioSessionMetaKey.nextUniqueKey();
-
-    /** Message tracker meta for session. */
-    private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
-
-    /**
-     * Default local port range (value is <tt>100</tt>).
-     * See {@link #setLocalPortRange(int)} for details.
-     */
-    public static final int DFLT_PORT_RANGE = 100;
-
-    /** Default value for {@code TCP_NODELAY} socket option (value is <tt>true</tt>). */
-    public static final boolean DFLT_TCP_NODELAY = true;
-
-    /** Server listener. */
-    private final GridNioServerListener<HadoopMessage> srvLsnr =
-        new GridNioServerListenerAdapter<HadoopMessage>() {
-            @Override public void onConnected(GridNioSession ses) {
-                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
-
-                assert desc != null : "Received connected notification without finished handshake: " + ses;
-            }
-
-            /** {@inheritDoc} */
-            @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
-                if (log.isDebugEnabled())
-                    log.debug("Closed connection for session: " + ses);
-
-                if (e != null)
-                    U.error(log, "Session disconnected due to exception: " + ses, e);
-
-                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
-
-                if (desc != null) {
-                    GridHadoopCommunicationClient rmv = clients.remove(desc.processId());
-
-                    if (rmv != null)
-                        rmv.forceClose();
-                }
-
-                GridHadoopMessageListener lsnr0 = lsnr;
-
-                if (lsnr0 != null)
-                    // Notify listener about connection close.
-                    lsnr0.onConnectionLost(desc);
-            }
-
-            /** {@inheritDoc} */
-            @Override public void onMessage(GridNioSession ses, HadoopMessage msg) {
-                notifyListener(ses.<HadoopProcessDescriptor>meta(PROCESS_META), msg);
-
-                if (msgQueueLimit > 0) {
-                    GridNioMessageTracker tracker = ses.meta(TRACKER_META);
-
-                    assert tracker != null : "Missing tracker for limited message queue: " + ses;
-
-                    tracker.run();
-                }
-            }
-        };
-
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Local process descriptor. */
-    private HadoopProcessDescriptor locProcDesc;
-
-    /** Marshaller. */
-    private Marshaller marsh;
-
-    /** Message notification executor service. */
-    private ExecutorService execSvc;
-
-    /** Grid name. */
-    private String gridName;
-
-    /** Complex variable that represents this node IP address. */
-    private volatile InetAddress locHost;
-
-    /** Local port which node uses. */
-    private int locPort = DFLT_PORT;
-
-    /** Local port range. */
-    private int locPortRange = DFLT_PORT_RANGE;
-
-    /** Local port which node uses to accept shared memory connections. */
-    private int shmemPort = -1;
-
-    /** Allocate direct buffer or heap buffer. */
-    private boolean directBuf = true;
-
-    /** Connect timeout. */
-    private long connTimeout = DFLT_CONN_TIMEOUT;
-
-    /** Maximum connect timeout. */
-    private long maxConnTimeout = DFLT_MAX_CONN_TIMEOUT;
-
-    /** Reconnect attempts count. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    private int reconCnt = DFLT_RECONNECT_CNT;
-
-    /** Socket send buffer. */
-    private int sockSndBuf;
-
-    /** Socket receive buffer. */
-    private int sockRcvBuf;
-
-    /** Message queue limit. */
-    private int msgQueueLimit = DFLT_MSG_QUEUE_LIMIT;
-
-    /** NIO server. */
-    private GridNioServer<HadoopMessage> nioSrvr;
-
-    /** Shared memory server. */
-    private IpcSharedMemoryServerEndpoint shmemSrv;
-
-    /** {@code TCP_NODELAY} option value for created sockets. */
-    private boolean tcpNoDelay = DFLT_TCP_NODELAY;
-
-    /** Shared memory accept worker. */
-    private ShmemAcceptWorker shmemAcceptWorker;
-
-    /** Shared memory workers. */
-    private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
-
-    /** Clients. */
-    private final ConcurrentMap<UUID, GridHadoopCommunicationClient> clients = GridConcurrentFactory.newMap();
-
-    /** Message listener. */
-    private volatile GridHadoopMessageListener lsnr;
-
-    /** Bound port. */
-    private int boundTcpPort = -1;
-
-    /** Bound port for shared memory server. */
-    private int boundTcpShmemPort = -1;
-
-    /** Count of selectors to use in TCP server. */
-    private int selectorsCnt = DFLT_SELECTORS_CNT;
-
-    /** Local node ID message. */
-    private ProcessHandshakeMessage locIdMsg;
-
-    /** Locks. */
-    private final GridKeyLock locks = new GridKeyLock();
-
-    /**
-     * @param parentNodeId Parent node ID.
-     * @param procId Process ID.
-     * @param marsh Marshaller to use.
-     * @param log Logger.
-     * @param execSvc Executor service for message notification.
-     * @param gridName Grid name.
-     */
-    public GridHadoopExternalCommunication(
-        UUID parentNodeId,
-        UUID procId,
-        Marshaller marsh,
-        IgniteLogger log,
-        ExecutorService execSvc,
-        String gridName
-    ) {
-        locProcDesc = new HadoopProcessDescriptor(parentNodeId, procId);
-
-        this.marsh = marsh;
-        this.log = log.getLogger(GridHadoopExternalCommunication.class);
-        this.execSvc = execSvc;
-        this.gridName = gridName;
-    }
-
-    /**
-     * Sets local port for socket binding.
-     * <p>
-     * If not provided, default value is {@link #DFLT_PORT}.
-     *
-     * @param locPort Port number.
-     */
-    public void setLocalPort(int locPort) {
-        this.locPort = locPort;
-    }
-
-    /**
-     * Gets local port for socket binding.
-     *
-     * @return Local port.
-     */
-    public int getLocalPort() {
-        return locPort;
-    }
-
-    /**
-     * Sets local port range for local host ports (value must greater than or equal to <tt>0</tt>).
-     * If provided local port (see {@link #setLocalPort(int)}} is occupied,
-     * implementation will try to increment the port number for as long as it is less than
-     * initial value plus this range.
-     * <p>
-     * If port range value is <tt>0</tt>, then implementation will try bind only to the port provided by
-     * {@link #setLocalPort(int)} method and fail if binding to this port did not succeed.
-     * <p>
-     * Local port range is very useful during development when more than one grid nodes need to run
-     * on the same physical machine.
-     * <p>
-     * If not provided, default value is {@link #DFLT_PORT_RANGE}.
-     *
-     * @param locPortRange New local port range.
-     */
-    public void setLocalPortRange(int locPortRange) {
-        this.locPortRange = locPortRange;
-    }
-
-    /**
-     * @return Local port range.
-     */
-    public int getLocalPortRange() {
-        return locPortRange;
-    }
-
-    /**
-     * Sets local port to accept shared memory connections.
-     * <p>
-     * If set to {@code -1} shared memory communication will be disabled.
-     * <p>
-     * If not provided, shared memory is disabled.
-     *
-     * @param shmemPort Port number.
-     */
-    public void setSharedMemoryPort(int shmemPort) {
-        this.shmemPort = shmemPort;
-    }
-
-    /**
-     * Gets shared memory port to accept incoming connections.
-     *
-     * @return Shared memory port.
-     */
-    public int getSharedMemoryPort() {
-        return shmemPort;
-    }
-
-    /**
-     * Sets connect timeout used when establishing connection
-     * with remote nodes.
-     * <p>
-     * {@code 0} is interpreted as infinite timeout.
-     * <p>
-     * If not provided, default value is {@link #DFLT_CONN_TIMEOUT}.
-     *
-     * @param connTimeout Connect timeout.
-     */
-    public void setConnectTimeout(long connTimeout) {
-        this.connTimeout = connTimeout;
-    }
-
-    /**
-     * @return Connection timeout.
-     */
-    public long getConnectTimeout() {
-        return connTimeout;
-    }
-
-    /**
-     * Sets maximum connect timeout. If handshake is not established within connect timeout,
-     * then SPI tries to repeat handshake procedure with increased connect timeout.
-     * Connect timeout can grow till maximum timeout value,
-     * if maximum timeout value is reached then the handshake is considered as failed.
-     * <p>
-     * {@code 0} is interpreted as infinite timeout.
-     * <p>
-     * If not provided, default value is {@link #DFLT_MAX_CONN_TIMEOUT}.
-     *
-     * @param maxConnTimeout Maximum connect timeout.
-     */
-    public void setMaxConnectTimeout(long maxConnTimeout) {
-        this.maxConnTimeout = maxConnTimeout;
-    }
-
-    /**
-     * Gets maximum connection timeout.
-     *
-     * @return Maximum connection timeout.
-     */
-    public long getMaxConnectTimeout() {
-        return maxConnTimeout;
-    }
-
-    /**
-     * Sets maximum number of reconnect attempts used when establishing connection
-     * with remote nodes.
-     * <p>
-     * If not provided, default value is {@link #DFLT_RECONNECT_CNT}.
-     *
-     * @param reconCnt Maximum number of reconnection attempts.
-     */
-    public void setReconnectCount(int reconCnt) {
-        this.reconCnt = reconCnt;
-    }
-
-    /**
-     * @return Reconnect count.
-     */
-    public int getReconnectCount() {
-        return reconCnt;
-    }
-
-    /**
-     * Sets flag to allocate direct or heap buffer in SPI.
-     * If value is {@code true}, then SPI will use {@link ByteBuffer#allocateDirect(int)} call.
-     * Otherwise, SPI will use {@link ByteBuffer#allocate(int)} call.
-     * <p>
-     * If not provided, default value is {@code true}.
-     *
-     * @param directBuf Flag indicates to allocate direct or heap buffer in SPI.
-     */
-    public void setDirectBuffer(boolean directBuf) {
-        this.directBuf = directBuf;
-    }
-
-    /**
-     * @return Direct buffer flag.
-     */
-    public boolean isDirectBuffer() {
-        return directBuf;
-    }
-
-    /**
-     * Sets the count of selectors te be used in TCP server.
-     * <p/>
-     * If not provided, default value is {@link #DFLT_SELECTORS_CNT}.
-     *
-     * @param selectorsCnt Selectors count.
-     */
-    public void setSelectorsCount(int selectorsCnt) {
-        this.selectorsCnt = selectorsCnt;
-    }
-
-    /**
-     * @return Number of selectors to use.
-     */
-    public int getSelectorsCount() {
-        return selectorsCnt;
-    }
-
-    /**
-     * Sets value for {@code TCP_NODELAY} socket option. Each
-     * socket will be opened using provided value.
-     * <p>
-     * Setting this option to {@code true} disables Nagle's algorithm
-     * for socket decreasing latency and delivery time for small messages.
-     * <p>
-     * For systems that work under heavy network load it is advisable to
-     * set this value to {@code false}.
-     * <p>
-     * If not provided, default value is {@link #DFLT_TCP_NODELAY}.
-     *
-     * @param tcpNoDelay {@code True} to disable TCP delay.
-     */
-    public void setTcpNoDelay(boolean tcpNoDelay) {
-        this.tcpNoDelay = tcpNoDelay;
-    }
-
-    /**
-     * @return {@code TCP_NO_DELAY} flag.
-     */
-    public boolean isTcpNoDelay() {
-        return tcpNoDelay;
-    }
-
-    /**
-     * Sets receive buffer size for sockets created or accepted by this SPI.
-     * <p>
-     * If not provided, default is {@code 0} which leaves buffer unchanged after
-     * socket creation (OS defaults).
-     *
-     * @param sockRcvBuf Socket receive buffer size.
-     */
-    public void setSocketReceiveBuffer(int sockRcvBuf) {
-        this.sockRcvBuf = sockRcvBuf;
-    }
-
-    /**
-     * @return Socket receive buffer size.
-     */
-    public int getSocketReceiveBuffer() {
-        return sockRcvBuf;
-    }
-
-    /**
-     * Sets send buffer size for sockets created or accepted by this SPI.
-     * <p>
-     * If not provided, default is {@code 0} which leaves the buffer unchanged
-     * after socket creation (OS defaults).
-     *
-     * @param sockSndBuf Socket send buffer size.
-     */
-    public void setSocketSendBuffer(int sockSndBuf) {
-        this.sockSndBuf = sockSndBuf;
-    }
-
-    /**
-     * @return Socket send buffer size.
-     */
-    public int getSocketSendBuffer() {
-        return sockSndBuf;
-    }
-
-    /**
-     * Sets message queue limit for incoming and outgoing messages.
-     * <p>
-     * When set to positive number send queue is limited to the configured value.
-     * {@code 0} disables the size limitations.
-     * <p>
-     * If not provided, default is {@link #DFLT_MSG_QUEUE_LIMIT}.
-     *
-     * @param msgQueueLimit Send queue size limit.
-     */
-    public void setMessageQueueLimit(int msgQueueLimit) {
-        this.msgQueueLimit = msgQueueLimit;
-    }
-
-    /**
-     * @return Message queue size limit.
-     */
-    public int getMessageQueueLimit() {
-        return msgQueueLimit;
-    }
-
-    /**
-     * Sets Hadoop communication message listener.
-     *
-     * @param lsnr Message listener.
-     */
-    public void setListener(GridHadoopMessageListener lsnr) {
-        this.lsnr = lsnr;
-    }
-
-    /**
-     * @return Outbound message queue size.
-     */
-    public int getOutboundMessagesQueueSize() {
-        return nioSrvr.outboundMessagesQueueSize();
-    }
-
-    /**
-     * Starts communication.
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    public void start() throws IgniteCheckedException {
-        try {
-            locHost = U.getLocalHost();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to initialize local address.", e);
-        }
-
-        try {
-            shmemSrv = resetShmemServer();
-        }
-        catch (IgniteCheckedException e) {
-            U.warn(log, "Failed to start shared memory communication server.", e);
-        }
-
-        try {
-            // This method potentially resets local port to the value
-            // local node was bound to.
-            nioSrvr = resetNioServer();
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Failed to initialize TCP server: " + locHost, e);
-        }
-
-        locProcDesc.address(locHost.getHostAddress());
-        locProcDesc.sharedMemoryPort(boundTcpShmemPort);
-        locProcDesc.tcpPort(boundTcpPort);
-
-        locIdMsg = new ProcessHandshakeMessage(locProcDesc);
-
-        if (shmemSrv != null) {
-            shmemAcceptWorker = new ShmemAcceptWorker(shmemSrv);
-
-            new IgniteThread(shmemAcceptWorker).start();
-        }
-
-        nioSrvr.start();
-    }
-
-    /**
-     * Gets local process descriptor.
-     *
-     * @return Local process descriptor.
-     */
-    public HadoopProcessDescriptor localProcessDescriptor() {
-        return locProcDesc;
-    }
-
-    /**
-     * Gets filters used by communication.
-     *
-     * @return Filters array.
-     */
-    private GridNioFilter[] filters() {
-        return new GridNioFilter[] {
-            new GridNioAsyncNotifyFilter(gridName, execSvc, log),
-            new HandshakeAndBackpressureFilter(),
-            new GridHadoopMarshallerFilter(marsh),
-            new GridNioCodecFilter(new GridBufferedParser(directBuf, ByteOrder.nativeOrder()), log, false)
-        };
-    }
-
-    /**
-     * Recreates tpcSrvr socket instance.
-     *
-     * @return Server instance.
-     * @throws IgniteCheckedException Thrown if it's not possible to create server.
-     */
-    private GridNioServer<HadoopMessage> resetNioServer() throws IgniteCheckedException {
-        if (boundTcpPort >= 0)
-            throw new IgniteCheckedException("Tcp NIO server was already created on port " + boundTcpPort);
-
-        IgniteCheckedException lastEx = null;
-
-        // If configured TCP port is busy, find first available in range.
-        for (int port = locPort; port < locPort + locPortRange; port++) {
-            try {
-                GridNioServer<HadoopMessage> srvr =
-                    GridNioServer.<HadoopMessage>builder()
-                        .address(locHost)
-                        .port(port)
-                        .listener(srvLsnr)
-                        .logger(log.getLogger(GridNioServer.class))
-                        .selectorCount(selectorsCnt)
-                        .gridName(gridName)
-                        .tcpNoDelay(tcpNoDelay)
-                        .directBuffer(directBuf)
-                        .byteOrder(ByteOrder.nativeOrder())
-                        .socketSendBufferSize(sockSndBuf)
-                        .socketReceiveBufferSize(sockRcvBuf)
-                        .sendQueueLimit(msgQueueLimit)
-                        .directMode(false)
-                        .filters(filters())
-                        .build();
-
-                boundTcpPort = port;
-
-                // Ack Port the TCP server was bound to.
-                if (log.isInfoEnabled())
-                    log.info("Successfully bound to TCP port [port=" + boundTcpPort +
-                        ", locHost=" + locHost + ']');
-
-                return srvr;
-            }
-            catch (IgniteCheckedException e) {
-                lastEx = e;
-
-                if (log.isDebugEnabled())
-                    log.debug("Failed to bind to local port (will try next port within range) [port=" + port +
-                        ", locHost=" + locHost + ']');
-            }
-        }
-
-        // If free port wasn't found.
-        throw new IgniteCheckedException("Failed to bind to any port within range [startPort=" + locPort +
-            ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
-    }
-
-    /**
-     * Creates new shared memory communication server.
-     * @return Server.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable private IpcSharedMemoryServerEndpoint resetShmemServer() throws IgniteCheckedException {
-        if (boundTcpShmemPort >= 0)
-            throw new IgniteCheckedException("Shared memory server was already created on port " + boundTcpShmemPort);
-
-        if (shmemPort == -1 || U.isWindows())
-            return null;
-
-        IgniteCheckedException lastEx = null;
-
-        // If configured TCP port is busy, find first available in range.
-        for (int port = shmemPort; port < shmemPort + locPortRange; port++) {
-            try {
-                IpcSharedMemoryServerEndpoint srv = new IpcSharedMemoryServerEndpoint(
-                    log.getLogger(IpcSharedMemoryServerEndpoint.class),
-                    locProcDesc.processId(), gridName);
-
-                srv.setPort(port);
-
-                srv.omitOutOfResourcesWarning(true);
-
-                srv.start();
-
-                boundTcpShmemPort = port;
-
-                // Ack Port the TCP server was bound to.
-                if (log.isInfoEnabled())
-                    log.info("Successfully bound shared memory communication to TCP port [port=" + boundTcpShmemPort +
-                        ", locHost=" + locHost + ']');
-
-                return srv;
-            }
-            catch (IgniteCheckedException e) {
-                lastEx = e;
-
-                if (log.isDebugEnabled())
-                    log.debug("Failed to bind to local port (will try next port within range) [port=" + port +
-                        ", locHost=" + locHost + ']');
-            }
-        }
-
-        // If free port wasn't found.
-        throw new IgniteCheckedException("Failed to bind shared memory communication to any port within range [startPort=" +
-            locPort + ", portRange=" + locPortRange + ", locHost=" + locHost + ']', lastEx);
-    }
-
-    /**
-     * Stops the server.
-     *
-     * @throws IgniteCheckedException
-     */
-    public void stop() throws IgniteCheckedException {
-        // Stop TCP server.
-        if (nioSrvr != null)
-            nioSrvr.stop();
-
-        U.cancel(shmemAcceptWorker);
-        U.join(shmemAcceptWorker, log);
-
-        U.cancel(shmemWorkers);
-        U.join(shmemWorkers, log);
-
-        shmemWorkers.clear();
-
-        // Force closing on stop (safety).
-        for (GridHadoopCommunicationClient client : clients.values())
-            client.forceClose();
-
-        // Clear resources.
-        nioSrvr = null;
-
-        boundTcpPort = -1;
-    }
-
-    /**
-     * Sends message to Hadoop process.
-     *
-     * @param desc
-     * @param msg
-     * @throws IgniteCheckedException
-     */
-    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws
-        IgniteCheckedException {
-        assert desc != null;
-        assert msg != null;
-
-        if (log.isTraceEnabled())
-            log.trace("Sending message to Hadoop process [desc=" + desc + ", msg=" + msg + ']');
-
-        GridHadoopCommunicationClient client = null;
-
-        boolean closeOnRelease = true;
-
-        try {
-            client = reserveClient(desc);
-
-            client.sendMessage(desc, msg);
-
-            closeOnRelease = false;
-        }
-        finally {
-            if (client != null) {
-                if (closeOnRelease) {
-                    client.forceClose();
-
-                    clients.remove(desc.processId(), client);
-                }
-                else
-                    client.release();
-            }
-        }
-    }
-
-    /**
-     * Returns existing or just created client to node.
-     *
-     * @param desc Node to which client should be open.
-     * @return The existing or just created client.
-     * @throws IgniteCheckedException Thrown if any exception occurs.
-     */
-    private GridHadoopCommunicationClient reserveClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
-        assert desc != null;
-
-        UUID procId = desc.processId();
-
-        while (true) {
-            GridHadoopCommunicationClient client = clients.get(procId);
-
-            if (client == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Did not find client for remote process [locProcDesc=" + locProcDesc + ", desc=" +
-                        desc + ']');
-
-                // Do not allow concurrent connects.
-                Object sync = locks.lock(procId);
-
-                try {
-                    client = clients.get(procId);
-
-                    if (client == null) {
-                        GridHadoopCommunicationClient old = clients.put(procId, client = createNioClient(desc));
-
-                        assert old == null;
-                    }
-                }
-                finally {
-                    locks.unlock(procId, sync);
-                }
-
-                assert client != null;
-            }
-
-            if (client.reserve())
-                return client;
-            else
-                // Client has just been closed by idle worker. Help it and try again.
-                clients.remove(procId, client);
-        }
-    }
-
-    /**
-     * @param desc Process descriptor.
-     * @return Client.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable protected GridHadoopCommunicationClient createNioClient(HadoopProcessDescriptor desc)
-        throws  IgniteCheckedException {
-        assert desc != null;
-
-        int shmemPort = desc.sharedMemoryPort();
-
-        // If remote node has shared memory server enabled and has the same set of MACs
-        // then we are likely to run on the same host and shared memory communication could be tried.
-        if (shmemPort != -1 && locProcDesc.parentNodeId().equals(desc.parentNodeId())) {
-            try {
-                return createShmemClient(desc, shmemPort);
-            }
-            catch (IgniteCheckedException e) {
-                if (e.hasCause(IpcOutOfSystemResourcesException.class))
-                    // Has cause or is itself the IpcOutOfSystemResourcesException.
-                    LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
-                else if (log.isDebugEnabled())
-                    log.debug("Failed to establish shared memory connection with local hadoop process: " +
-                        desc);
-            }
-        }
-
-        return createTcpClient(desc);
-    }
-
-    /**
-     * @param desc Process descriptor.
-     * @param port Port.
-     * @return Client.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable protected GridHadoopCommunicationClient createShmemClient(HadoopProcessDescriptor desc, int port)
-        throws IgniteCheckedException {
-        int attempt = 1;
-
-        int connectAttempts = 1;
-
-        long connTimeout0 = connTimeout;
-
-        while (true) {
-            IpcEndpoint clientEndpoint;
-
-            try {
-                clientEndpoint = new IpcSharedMemoryClientEndpoint(port, (int)connTimeout, log);
-            }
-            catch (IgniteCheckedException e) {
-                // Reconnect for the second time, if connection is not established.
-                if (connectAttempts < 2 && X.hasCause(e, ConnectException.class)) {
-                    connectAttempts++;
-
-                    continue;
-                }
-
-                throw e;
-            }
-
-            GridHadoopCommunicationClient client = null;
-
-            try {
-                ShmemWorker worker = new ShmemWorker(clientEndpoint, false);
-
-                shmemWorkers.add(worker);
-
-                GridNioSession ses = worker.session();
-
-                HandshakeFinish fin = new HandshakeFinish();
-
-                // We are in lock, it is safe to get session and attach
-                ses.addMeta(HANDSHAKE_FINISH_META, fin);
-
-                client = new GridHadoopTcpNioCommunicationClient(ses);
-
-                new IgniteThread(worker).start();
-
-                fin.await(connTimeout0);
-            }
-            catch (GridHadoopHandshakeTimeoutException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Handshake timed out (will retry with increased timeout) [timeout=" + connTimeout0 +
-                        ", err=" + e.getMessage() + ", client=" + client + ']');
-
-                if (client != null)
-                    client.forceClose();
-
-                if (attempt == reconCnt || connTimeout0 > maxConnTimeout) {
-                    if (log.isDebugEnabled())
-                        log.debug("Handshake timedout (will stop attempts to perform the handshake) " +
-                            "[timeout=" + connTimeout0 + ", maxConnTimeout=" + maxConnTimeout +
-                            ", attempt=" + attempt + ", reconCnt=" + reconCnt +
-                            ", err=" + e.getMessage() + ", client=" + client + ']');
-
-                    throw e;
-                }
-                else {
-                    attempt++;
-
-                    connTimeout0 *= 2;
-
-                    continue;
-                }
-            }
-            catch (RuntimeException | Error e) {
-                if (log.isDebugEnabled())
-                    log.debug(
-                        "Caught exception (will close client) [err=" + e.getMessage() + ", client=" + client + ']');
-
-                if (client != null)
-                    client.forceClose();
-
-                throw e;
-            }
-
-            return client;
-        }
-    }
-
-    /**
-     * Establish TCP connection to remote hadoop process and returns client.
-     *
-     * @param desc Process descriptor.
-     * @return Client.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected GridHadoopCommunicationClient createTcpClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
-        String addr = desc.address();
-
-        int port = desc.tcpPort();
-
-        if (log.isDebugEnabled())
-            log.debug("Trying to connect to remote process [locProcDesc=" + locProcDesc + ", desc=" + desc + ']');
-
-        boolean conn = false;
-        GridHadoopTcpNioCommunicationClient client = null;
-        IgniteCheckedException errs = null;
-
-        int connectAttempts = 1;
-
-        long connTimeout0 = connTimeout;
-
-        int attempt = 1;
-
-        while (!conn) { // Reconnection on handshake timeout.
-            try {
-                SocketChannel ch = SocketChannel.open();
-
-                ch.configureBlocking(true);
-
-                ch.socket().setTcpNoDelay(tcpNoDelay);
-                ch.socket().setKeepAlive(true);
-
-                if (sockRcvBuf > 0)
-                    ch.socket().setReceiveBufferSize(sockRcvBuf);
-
-                if (sockSndBuf > 0)
-                    ch.socket().setSendBufferSize(sockSndBuf);
-
-                ch.socket().connect(new InetSocketAddress(addr, port), (int)connTimeout);
-
-                HandshakeFinish fin = new HandshakeFinish();
-
-                GridNioSession ses = nioSrvr.createSession(ch, F.asMap(HANDSHAKE_FINISH_META, fin)).get();
-
-                client = new GridHadoopTcpNioCommunicationClient(ses);
-
-                if (log.isDebugEnabled())
-                    log.debug("Waiting for handshake finish for client: " + client);
-
-                fin.await(connTimeout0);
-
-                conn = true;
-            }
-            catch (GridHadoopHandshakeTimeoutException e) {
-                if (client != null) {
-                    client.forceClose();
-
-                    client = null;
-                }
-
-                if (log.isDebugEnabled())
-                    log.debug(
-                        "Handshake timedout (will retry with increased timeout) [timeout=" + connTimeout0 +
-                            ", desc=" + desc + ", port=" + port + ", err=" + e + ']');
-
-                if (attempt == reconCnt || connTimeout0 > maxConnTimeout) {
-                    if (log.isDebugEnabled())
-                        log.debug("Handshake timed out (will stop attempts to perform the handshake) " +
-                            "[timeout=" + connTimeout0 + ", maxConnTimeout=" + maxConnTimeout +
-                            ", attempt=" + attempt + ", reconCnt=" + reconCnt +
-                            ", err=" + e.getMessage() + ", addr=" + addr + ']');
-
-                    if (errs == null)
-                        errs = new IgniteCheckedException("Failed to connect to remote Hadoop process " +
-                            "(is process still running?) [desc=" + desc + ", addrs=" + addr + ']');
-
-                    errs.addSuppressed(e);
-
-                    break;
-                }
-                else {
-                    attempt++;
-
-                    connTimeout0 *= 2;
-
-                    // Continue loop.
-                }
-            }
-            catch (Exception e) {
-                if (client != null) {
-                    client.forceClose();
-
-                    client = null;
-                }
-
-                if (log.isDebugEnabled())
-                    log.debug("Client creation failed [addr=" + addr + ", port=" + port +
-                        ", err=" + e + ']');
-
-                if (X.hasCause(e, SocketTimeoutException.class))
-                    LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
-                        "configuration property) [addr=" + addr + ", port=" + port + ']');
-
-                if (errs == null)
-                    errs = new IgniteCheckedException("Failed to connect to remote Hadoop process (is process still running?) " +
-                        "[desc=" + desc + ", addrs=" + addr + ']');
-
-                errs.addSuppressed(e);
-
-                // Reconnect for the second time, if connection is not established.
-                if (connectAttempts < 2 &&
-                    (e instanceof ConnectException || X.hasCause(e, ConnectException.class))) {
-                    connectAttempts++;
-
-                    continue;
-                }
-
-                break;
-            }
-        }
-
-        if (client == null) {
-            assert errs != null;
-
-            if (X.hasCause(errs, ConnectException.class))
-                LT.warn(log, null, "Failed to connect to a remote Hadoop process (is process still running?). " +
-                    "Make sure operating system firewall is disabled on local and remote host) " +
-                    "[addrs=" + addr + ", port=" + port + ']');
-
-            throw errs;
-        }
-
-        if (log.isDebugEnabled())
-            log.debug("Created client: " + client);
-
-        return client;
-    }
-
-    /**
-     * @param desc Sender process descriptor.
-     * @param msg Communication message.
-     */
-    protected void notifyListener(HadoopProcessDescriptor desc, HadoopMessage msg) {
-        GridHadoopMessageListener lsnr = this.lsnr;
-
-        if (lsnr != null)
-            // Notify listener of a new message.
-            lsnr.onMessageReceived(desc, msg);
-        else if (log.isDebugEnabled())
-            log.debug("Received communication message without any registered listeners (will ignore) " +
-                "[senderProcDesc=" + desc + ", msg=" + msg + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopExternalCommunication.class, this);
-    }
-
-    /**
-     * This worker takes responsibility to shut the server down when stopping,
-     * No other thread shall stop passed server.
-     */
-    private class ShmemAcceptWorker extends GridWorker {
-        /** */
-        private final IpcSharedMemoryServerEndpoint srv;
-
-        /**
-         * @param srv Server.
-         */
-        ShmemAcceptWorker(IpcSharedMemoryServerEndpoint srv) {
-            super(gridName, "shmem-communication-acceptor", log);
-
-            this.srv = srv;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            try {
-                while (!Thread.interrupted()) {
-                    ShmemWorker e = new ShmemWorker(srv.accept(), true);
-
-                    shmemWorkers.add(e);
-
-                    new IgniteThread(e).start();
-                }
-            }
-            catch (IgniteCheckedException e) {
-                if (!isCancelled())
-                    U.error(log, "Shmem server failed.", e);
-            }
-            finally {
-                srv.close();
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            super.cancel();
-
-            srv.close();
-        }
-    }
-
-    /**
-     *
-     */
-    private class ShmemWorker extends GridWorker {
-        /** */
-        private final IpcEndpoint endpoint;
-
-        /** Adapter. */
-        private GridHadoopIpcToNioAdapter<HadoopMessage> adapter;
-
-        /**
-         * @param endpoint Endpoint.
-         */
-        private ShmemWorker(IpcEndpoint endpoint, boolean accepted) {
-            super(gridName, "shmem-worker", log);
-
-            this.endpoint = endpoint;
-
-            adapter = new GridHadoopIpcToNioAdapter<>(
-                GridHadoopExternalCommunication.this.log,
-                endpoint,
-                accepted,
-                srvLsnr,
-                filters());
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            try {
-                adapter.serve();
-            }
-            finally {
-                shmemWorkers.remove(this);
-
-                endpoint.close();
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            super.cancel();
-
-            endpoint.close();
-        }
-
-        /** @{@inheritDoc} */
-        @Override protected void cleanup() {
-            super.cleanup();
-
-            endpoint.close();
-        }
-
-        /** @{@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ShmemWorker.class, this);
-        }
-
-        /**
-         * @return NIO session for this worker.
-         */
-        public GridNioSession session() {
-            return adapter.session();
-        }
-    }
-
-    /**
-     *
-     */
-    private static class HandshakeFinish {
-        /** Await latch. */
-        private CountDownLatch latch = new CountDownLatch(1);
-
-        /**
-         * Finishes handshake.
-         */
-        public void finish() {
-            latch.countDown();
-        }
-
-        /**
-         * @param time Time to wait.
-         * @throws GridHadoopHandshakeTimeoutException If failed to wait.
-         */
-        public void await(long time) throws GridHadoopHandshakeTimeoutException {
-            try {
-                if (!latch.await(time, TimeUnit.MILLISECONDS))
-                    throw new GridHadoopHandshakeTimeoutException("Failed to wait for handshake to finish [timeout=" +
-                        time + ']');
-            }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-
-                throw new GridHadoopHandshakeTimeoutException("Failed to wait for handshake to finish (thread was " +
-                    "interrupted) [timeout=" + time + ']', e);
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private class HandshakeAndBackpressureFilter extends GridNioFilterAdapter {
-        /**
-         * Assigns filter name to a filter.
-         */
-        protected HandshakeAndBackpressureFilter() {
-            super("HadoopHandshakeFilter");
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionOpened(final GridNioSession ses) throws IgniteCheckedException {
-            if (ses.accepted()) {
-                if (log.isDebugEnabled())
-                    log.debug("Accepted connection, initiating handshake: " + ses);
-
-                // Server initiates handshake.
-                ses.send(locIdMsg).listenAsync(new CI1<GridNioFuture<?>>() {
-                    @Override public void apply(GridNioFuture<?> fut) {
-                        try {
-                            // Make sure there were no errors.
-                            fut.get();
-                        }
-                        catch (IgniteCheckedException | IOException e) {
-                            log.warning("Failed to send handshake message, will close session: " + ses, e);
-
-                            ses.close();
-                        }
-                    }
-                });
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionClosed(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
-            proceedExceptionCaught(ses, ex);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-            if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
-                log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
-
-            return proceedSessionWrite(ses, msg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
-            HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
-
-            UUID rmtProcId = desc == null ? null : desc.processId();
-
-            if (rmtProcId == null) {
-                if (!(msg instanceof ProcessHandshakeMessage)) {
-                    log.warning("Invalid handshake message received, will close connection [ses=" + ses +
-                        ", msg=" + msg + ']');
-
-                    ses.close();
-
-                    return;
-                }
-
-                ProcessHandshakeMessage nId = (ProcessHandshakeMessage)msg;
-
-                if (log.isDebugEnabled())
-                    log.debug("Received handshake message [ses=" + ses + ", msg=" + msg + ']');
-
-                ses.addMeta(PROCESS_META, nId.processDescriptor());
-
-                if (!ses.accepted())
-                    // Send handshake reply.
-                    ses.send(locIdMsg);
-                else {
-                    //
-                    rmtProcId = nId.processDescriptor().processId();
-
-                    if (log.isDebugEnabled())
-                        log.debug("Finished handshake with remote client: " + ses);
-
-                    Object sync = locks.tryLock(rmtProcId);
-
-                    if (sync != null) {
-                        try {
-                            if (clients.get(rmtProcId) == null) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Will reuse session for descriptor: " + rmtProcId);
-
-                                // Handshake finished flag is true.
-                                clients.put(rmtProcId, new GridHadoopTcpNioCommunicationClient(ses));
-                            }
-                            else {
-                                if (log.isDebugEnabled())
-                                    log.debug("Will not reuse client as another already exists [locProcDesc=" +
-                                        locProcDesc + ", desc=" + desc + ']');
-                            }
-                        }
-                        finally {
-                            locks.unlock(rmtProcId, sync);
-                        }
-                    }
-                    else {
-                        if (log.isDebugEnabled())
-                            log.debug("Concurrent connection is being established, will not reuse client session [" +
-                                "locProcDesc=" + locProcDesc + ", desc=" + desc + ']');
-                    }
-                }
-
-                if (log.isDebugEnabled())
-                    log.debug("Handshake is finished for session [ses=" + ses + ", locProcDesc=" + locProcDesc + ']');
-
-                HandshakeFinish to = ses.meta(HANDSHAKE_FINISH_META);
-
-                if (to != null)
-                    to.finish();
-
-                // Notify session opened (both parties).
-                proceedSessionOpened(ses);
-            }
-            else {
-                if (msgQueueLimit > 0) {
-                    GridNioMessageTracker tracker = ses.meta(TRACKER_META);
-
-                    if (tracker == null) {
-                        GridNioMessageTracker old = ses.addMeta(TRACKER_META, tracker =
-                            new GridNioMessageTracker(ses, msgQueueLimit));
-
-                        assert old == null;
-                    }
-
-                    tracker.onMessageReceived();
-                }
-
-                proceedMessageReceived(ses, msg);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) throws IgniteCheckedException {
-            return proceedSessionClose(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionIdleTimeout(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionWriteTimeout(ses);
-        }
-    }
-
-    /**
-     * Process ID message.
-     */
-    @SuppressWarnings("PublicInnerClass")
-    public static class ProcessHandshakeMessage implements HadoopMessage {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Node ID. */
-        private HadoopProcessDescriptor procDesc;
-
-        /** */
-        public ProcessHandshakeMessage() {
-            // No-op.
-        }
-
-        /**
-         * @param procDesc Process descriptor.
-         */
-        private ProcessHandshakeMessage(HadoopProcessDescriptor procDesc) {
-            this.procDesc = procDesc;
-        }
-
-        /**
-         * @return Process ID.
-         */
-        public HadoopProcessDescriptor processDescriptor() {
-            return procDesc;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            out.writeObject(procDesc);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            procDesc = (HadoopProcessDescriptor)in.readObject();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ProcessHandshakeMessage.class, this);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopHandshakeTimeoutException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopHandshakeTimeoutException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopHandshakeTimeoutException.java
deleted file mode 100644
index e001dc9..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopHandshakeTimeoutException.java
+++ /dev/null
@@ -1,42 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.jetbrains.annotations.*;
-
-/** Internal exception class for proper timeout handling. */
-class GridHadoopHandshakeTimeoutException extends IgniteCheckedException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * @param msg Message.
-     */
-    GridHadoopHandshakeTimeoutException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * @param msg Message.
-     * @param cause Cause.
-     */
-    GridHadoopHandshakeTimeoutException(String msg, @Nullable Throwable cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopIpcToNioAdapter.java
deleted file mode 100644
index a39451d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopIpcToNioAdapter.java
+++ /dev/null
@@ -1,239 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.ipc.*;
-import org.apache.ignite.internal.util.nio.*;
-
-import java.io.*;
-import java.nio.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Allows to re-use existing {@link GridNioFilter}s on IPC (specifically shared memory IPC)
- * communications.
- *
- * Note that this class consumes an entire thread inside {@link #serve()} method
- * in order to serve one {@link org.apache.ignite.internal.util.ipc.IpcEndpoint}.
- */
-public class GridHadoopIpcToNioAdapter<T> {
-    /** */
-    private final IpcEndpoint endp;
-
-    /** */
-    private final GridNioFilterChain<T> chain;
-
-    /** */
-    private final GridNioSessionImpl ses;
-
-    /** */
-    private final AtomicReference<CountDownLatch> latchRef = new AtomicReference<>();
-
-    /** */
-    private final ByteBuffer writeBuf;
-
-    /**
-     * @param log Log.
-     * @param endp Endpoint.
-     * @param lsnr Listener.
-     * @param filters Filters.
-     */
-    public GridHadoopIpcToNioAdapter(IgniteLogger log, IpcEndpoint endp, boolean accepted,
-        GridNioServerListener<T> lsnr, GridNioFilter... filters) {
-        this.endp = endp;
-
-        chain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
-        ses = new GridNioSessionImpl(chain, null, null, accepted);
-
-        writeBuf = ByteBuffer.allocate(8 << 10);
-
-        writeBuf.order(ByteOrder.nativeOrder());
-    }
-
-    /**
-     * Serves given set of listeners repeatedly reading data from the endpoint.
-     *
-     * @throws InterruptedException If interrupted.
-     */
-    public void serve() throws InterruptedException {
-        try {
-            chain.onSessionOpened(ses);
-
-            InputStream in = endp.inputStream();
-
-            ByteBuffer readBuf = ByteBuffer.allocate(8 << 10);
-
-            readBuf.order(ByteOrder.nativeOrder());
-
-            assert readBuf.hasArray();
-
-            while (!Thread.interrupted()) {
-                int pos = readBuf.position();
-
-                int read = in.read(readBuf.array(), pos, readBuf.remaining());
-
-                if (read > 0) {
-                    readBuf.position(0);
-                    readBuf.limit(pos + read);
-
-                    chain.onMessageReceived(ses, readBuf);
-
-                    if (readBuf.hasRemaining())
-                        readBuf.compact();
-                    else
-                        readBuf.clear();
-
-                    CountDownLatch latch = latchRef.get();
-
-                    if (latch != null)
-                        latch.await();
-                }
-                else if (read < 0) {
-                    endp.close();
-
-                    break; // And close below.
-                }
-            }
-
-            // Assuming remote end closed connection - pushing event from head to tail.
-            chain.onSessionClosed(ses);
-        }
-        catch (Exception e) {
-            chain.onExceptionCaught(ses, new IgniteCheckedException("Failed to read from IPC endpoint.", e));
-        }
-    }
-
-    /**
-     * Gets dummy session for this adapter.
-     *
-     * @return Session.
-     */
-    public GridNioSession session() {
-        return ses;
-    }
-
-    /**
-     * Handles write events on chain.
-     *
-     * @param msg Buffer to send.
-     * @return Send result.
-     */
-    private GridNioFuture<?> send(ByteBuffer msg) {
-        assert writeBuf.hasArray();
-
-        try {
-            while (msg.hasRemaining()) {
-                writeBuf.clear();
-
-                writeBuf.put(msg);
-
-                endp.outputStream().write(writeBuf.array(), 0, writeBuf.position());
-            }
-        }
-        catch (IOException | IgniteCheckedException e) {
-            return new GridNioFinishedFuture<Object>(e);
-        }
-
-        return new GridNioFinishedFuture<>((Object)null);
-    }
-
-    /**
-     * Filter forwarding messages from chain's head to this server.
-     */
-    private class HeadFilter extends GridNioFilterAdapter {
-        /**
-         * Assigns filter name.
-         */
-        protected HeadFilter() {
-            super("HeadFilter");
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionOpened(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionClosed(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
-            proceedExceptionCaught(ses, ex);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
-            assert ses == GridHadoopIpcToNioAdapter.this.ses : "ses=" + ses +
-                ", this.ses=" + GridHadoopIpcToNioAdapter.this.ses;
-
-            return send((ByteBuffer)msg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
-            proceedMessageReceived(ses, msg);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onPauseReads(GridNioSession ses) throws IgniteCheckedException {
-            // This call should be synced externally to avoid races.
-            boolean b = latchRef.compareAndSet(null, new CountDownLatch(1));
-
-            assert b;
-
-            return new GridNioFinishedFuture<>(b);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onResumeReads(GridNioSession ses) throws IgniteCheckedException {
-            // This call should be synced externally to avoid races.
-            CountDownLatch latch = latchRef.getAndSet(null);
-
-            if (latch != null)
-                latch.countDown();
-
-            return new GridNioFinishedFuture<Object>(latch != null);
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) {
-            assert ses == GridHadoopIpcToNioAdapter.this.ses;
-
-            boolean closed = GridHadoopIpcToNioAdapter.this.ses.setClosed();
-
-            if (closed)
-                endp.close();
-
-            return new GridNioFinishedFuture<>(closed);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionIdleTimeout(ses);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
-            proceedSessionWriteTimeout(ses);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
deleted file mode 100644
index e9dfc92..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
+++ /dev/null
@@ -1,84 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.nio.*;
-import org.apache.ignite.marshaller.*;
-
-/**
- * Serialization filter.
- */
-public class GridHadoopMarshallerFilter extends GridNioFilterAdapter {
-    /** Marshaller. */
-    private Marshaller marshaller;
-
-    /**
-     * @param marshaller Marshaller to use.
-     */
-    public GridHadoopMarshallerFilter(Marshaller marshaller) {
-        super("GridHadoopMarshallerFilter");
-
-        this.marshaller = marshaller;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionOpened(GridNioSession ses) throws IgniteCheckedException {
-        proceedSessionOpened(ses);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionClosed(GridNioSession ses) throws IgniteCheckedException {
-        proceedSessionClosed(ses);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
-        proceedExceptionCaught(ses, ex);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
-
-        return proceedSessionWrite(ses, marshaller.marshal(msg));
-    }
-
-    @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        assert msg instanceof byte[];
-
-        // Always unmarshal with system classloader.
-        proceedMessageReceived(ses, marshaller.unmarshal((byte[])msg, null));
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridNioFuture<Boolean> onSessionClose(GridNioSession ses) throws IgniteCheckedException {
-        return proceedSessionClose(ses);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionIdleTimeout(GridNioSession ses) throws IgniteCheckedException {
-        proceedSessionIdleTimeout(ses);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
-        proceedSessionWriteTimeout(ses);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
deleted file mode 100644
index 6010a8d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
+++ /dev/null
@@ -1,39 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-
-/**
- * Hadoop communication message listener.
- */
-public interface GridHadoopMessageListener {
-    /**
-     * @param desc Process descriptor.
-     * @param msg Hadoop message.
-     */
-    public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg);
-
-    /**
-     * Called when connection to remote process was lost.
-     *
-     * @param desc Process descriptor.
-     */
-    public void onConnectionLost(HadoopProcessDescriptor desc);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
deleted file mode 100644
index a2b89a6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
+++ /dev/null
@@ -1,99 +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.taskexecutor.external.communication;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.nio.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Grid client for NIO server.
- */
-public class GridHadoopTcpNioCommunicationClient extends GridHadoopAbstractCommunicationClient {
-    /** Socket. */
-    private final GridNioSession ses;
-
-    /**
-     * Constructor for test purposes only.
-     */
-    public GridHadoopTcpNioCommunicationClient() {
-        ses = null;
-    }
-
-    /**
-     * @param ses Session.
-     */
-    public GridHadoopTcpNioCommunicationClient(GridNioSession ses) {
-        assert ses != null;
-
-        this.ses = ses;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean close() {
-        boolean res = super.close();
-
-        if (res)
-            ses.close();
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void forceClose() {
-        super.forceClose();
-
-        ses.close();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg)
-        throws IgniteCheckedException {
-        if (closed())
-            throw new IgniteCheckedException("Client was closed: " + this);
-
-        GridNioFuture<?> fut = ses.send(msg);
-
-        if (fut.isDone()) {
-            try {
-                fut.get();
-            }
-            catch (IOException e) {
-                throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getIdleTime() {
-        long now = U.currentTimeMillis();
-
-        // Session can be used for receiving and sending.
-        return Math.min(Math.min(now - ses.lastReceiveTime(), now - ses.lastSendScheduleTime()),
-            now - ses.lastSendTime());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopTcpNioCommunicationClient.class, this, super.toString());
-    }
-}


[26/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIo.java
deleted file mode 100644
index d07f34d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIo.java
+++ /dev/null
@@ -1,599 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.ipc.*;
-import org.apache.ignite.internal.util.ipc.shmem.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-/**
- * IO layer implementation based on blocking IPC streams.
- */
-@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-public class IgfsHadoopIpcIo implements IgfsHadoopIo {
-    /** Logger. */
-    private Log log;
-
-    /** Request futures map. */
-    private ConcurrentMap<Long, IgfsHadoopFuture> reqMap =
-        new ConcurrentHashMap8<>();
-
-    /** Request ID counter. */
-    private AtomicLong reqIdCnt = new AtomicLong();
-
-    /** Endpoint. */
-    private IpcEndpoint endpoint;
-
-    /** Endpoint output stream. */
-    private IgfsDataOutputStream out;
-
-    /** Protocol. */
-    private final IgfsMarshaller marsh;
-
-    /** Client reader thread. */
-    private Thread reader;
-
-    /** Lock for graceful shutdown. */
-    private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
-
-    /** Stopping flag. */
-    private volatile boolean stopping;
-
-    /** Server endpoint address. */
-    private final String endpointAddr;
-
-    /** Number of open file system sessions. */
-    private final AtomicInteger activeCnt = new AtomicInteger(1);
-
-    /** Event listeners. */
-    private final Collection<IgfsHadoopIpcIoListener> lsnrs =
-        new GridConcurrentHashSet<>();
-
-    /** Cached connections. */
-    private static final ConcurrentMap<String, IgfsHadoopIpcIo> ipcCache =
-        new ConcurrentHashMap8<>();
-
-    /** Striped lock that prevents multiple instance creation in {@link #get(Log, String)}. */
-    private static final GridStripedLock initLock = new GridStripedLock(32);
-
-    /**
-     * @param endpointAddr Endpoint.
-     * @param marsh Protocol.
-     * @param log Logger to use.
-     */
-    public IgfsHadoopIpcIo(String endpointAddr, IgfsMarshaller marsh, Log log) {
-        assert endpointAddr != null;
-        assert marsh != null;
-
-        this.endpointAddr = endpointAddr;
-        this.marsh = marsh;
-        this.log = log;
-    }
-
-    /**
-     * Returns a started and valid instance of this class
-     * for a given endpoint.
-     *
-     * @param log Logger to use for new instance.
-     * @param endpoint Endpoint string.
-     * @return New or existing cached instance, which is started and operational.
-     * @throws IOException If new instance was created but failed to start.
-     */
-    public static IgfsHadoopIpcIo get(Log log, String endpoint) throws IOException {
-        while (true) {
-            IgfsHadoopIpcIo clientIo = ipcCache.get(endpoint);
-
-            if (clientIo != null) {
-                if (clientIo.acquire())
-                    return clientIo;
-                else
-                    // If concurrent close.
-                    ipcCache.remove(endpoint, clientIo);
-            }
-            else {
-                Lock lock = initLock.getLock(endpoint);
-
-                lock.lock();
-
-                try {
-                    clientIo = ipcCache.get(endpoint);
-
-                    if (clientIo != null) { // Perform double check.
-                        if (clientIo.acquire())
-                            return clientIo;
-                        else
-                            // If concurrent close.
-                            ipcCache.remove(endpoint, clientIo);
-                    }
-
-                    // Otherwise try creating a new one.
-                    clientIo = new IgfsHadoopIpcIo(endpoint, new IgfsMarshaller(), log);
-
-                    try {
-                        clientIo.start();
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IOException(e.getMessage(), e);
-                    }
-
-                    IgfsHadoopIpcIo old = ipcCache.putIfAbsent(endpoint, clientIo);
-
-                    // Put in exclusive lock.
-                    assert old == null;
-
-                    return clientIo;
-                }
-                finally {
-                    lock.unlock();
-                }
-            }
-        }
-    }
-
-    /**
-     * Increases usage count for this instance.
-     *
-     * @return {@code true} if usage count is greater than zero.
-     */
-    private boolean acquire() {
-        while (true) {
-            int cnt = activeCnt.get();
-
-            if (cnt == 0) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO not acquired (count was 0): " + this);
-
-                return false;
-            }
-
-            // Need to make sure that no-one decremented count in between.
-            if (activeCnt.compareAndSet(cnt, cnt + 1)) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO acquired: " + this);
-
-                return true;
-            }
-        }
-    }
-
-    /**
-     * Releases this instance, decrementing usage count.
-     * <p>
-     * If usage count becomes zero, the instance is stopped
-     * and removed from cache.
-     */
-    public void release() {
-        while (true) {
-            int cnt = activeCnt.get();
-
-            if (cnt == 0) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO not released (count was 0): " + this);
-
-                return;
-            }
-
-            if (activeCnt.compareAndSet(cnt, cnt - 1)) {
-                if (cnt == 1) {
-                    ipcCache.remove(endpointAddr, this);
-
-                    if (log.isDebugEnabled())
-                        log.debug("IPC IO stopping as unused: " + this);
-
-                    stop();
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("IPC IO released: " + this);
-
-                return;
-            }
-        }
-    }
-
-    /**
-     * Closes this IO instance, removing it from cache.
-     */
-    public void forceClose() {
-        if (ipcCache.remove(endpointAddr, this))
-            stop();
-    }
-
-    /**
-     * Starts the IO.
-     *
-     * @throws IgniteCheckedException If failed to connect the endpoint.
-     */
-    private void start() throws IgniteCheckedException {
-        boolean success = false;
-
-        try {
-            endpoint = IpcEndpointFactory.connectEndpoint(
-                    endpointAddr, new GridLoggerProxy(new IgfsHadoopJclLogger(log), null, null, ""));
-
-            out = new IgfsDataOutputStream(new BufferedOutputStream(endpoint.outputStream()));
-
-            reader = new ReaderThread();
-
-            // Required for Hadoop 2.x
-            reader.setDaemon(true);
-
-            reader.start();
-
-            success = true;
-        }
-        catch (IgniteCheckedException e) {
-            IpcOutOfSystemResourcesException resEx = e.getCause(IpcOutOfSystemResourcesException.class);
-
-            if (resEx != null)
-                throw new IgniteCheckedException(IpcSharedMemoryServerEndpoint.OUT_OF_RESOURCES_MSG, resEx);
-
-            throw e;
-        }
-        finally {
-            if (!success)
-                stop();
-        }
-    }
-
-    /**
-     * Shuts down the IO. No send requests will be accepted anymore, all pending futures will be failed.
-     * Close listeners will be invoked as if connection is closed by server.
-     */
-    private void stop() {
-        close0(null);
-
-        if (reader != null) {
-            try {
-                U.interrupt(reader);
-                U.join(reader);
-
-                reader = null;
-            }
-            catch (IgniteInterruptedCheckedException ignored) {
-                Thread.currentThread().interrupt();
-
-                log.warn("Got interrupted while waiting for reader thread to shut down (will return).");
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addEventListener(IgfsHadoopIpcIoListener lsnr) {
-        if (!busyLock.readLock().tryLock()) {
-            lsnr.onClose();
-
-            return;
-        }
-
-        boolean invokeNow = false;
-
-        try {
-            invokeNow = stopping;
-
-            if (!invokeNow)
-                lsnrs.add(lsnr);
-        }
-        finally {
-            busyLock.readLock().unlock();
-
-            if (invokeNow)
-                lsnr.onClose();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeEventListener(IgfsHadoopIpcIoListener lsnr) {
-        lsnrs.remove(lsnr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException {
-        return send(msg, null, 0, 0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff,
-        int outLen) throws IgniteCheckedException {
-        assert outBuf == null || msg.command() == IgfsIpcCommand.READ_BLOCK;
-
-        if (!busyLock.readLock().tryLock())
-            throw new IgfsHadoopCommunicationException("Failed to send message (client is being concurrently " +
-                "closed).");
-
-        try {
-            if (stopping)
-                throw new IgfsHadoopCommunicationException("Failed to send message (client is being concurrently " +
-                    "closed).");
-
-            long reqId = reqIdCnt.getAndIncrement();
-
-            IgfsHadoopFuture<T> fut = new IgfsHadoopFuture<>();
-
-            fut.outputBuffer(outBuf);
-            fut.outputOffset(outOff);
-            fut.outputLength(outLen);
-            fut.read(msg.command() == IgfsIpcCommand.READ_BLOCK);
-
-            IgfsHadoopFuture oldFut = reqMap.putIfAbsent(reqId, fut);
-
-            assert oldFut == null;
-
-            if (log.isDebugEnabled())
-                log.debug("Sending IGFS message [reqId=" + reqId + ", msg=" + msg + ']');
-
-            byte[] hdr = IgfsMarshaller.createHeader(reqId, msg.command());
-
-            IgniteCheckedException err = null;
-
-            try {
-                synchronized (this) {
-                    marsh.marshall(msg, hdr, out);
-
-                    out.flush(); // Blocking operation + sometimes system call.
-                }
-            }
-            catch (IgniteCheckedException e) {
-                err = e;
-            }
-            catch (IOException e) {
-                err = new IgfsHadoopCommunicationException(e);
-            }
-
-            if (err != null) {
-                reqMap.remove(reqId, fut);
-
-                fut.onDone(err);
-            }
-
-            return fut;
-        }
-        finally {
-            busyLock.readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendPlain(IgfsMessage msg) throws IgniteCheckedException {
-        if (!busyLock.readLock().tryLock())
-            throw new IgfsHadoopCommunicationException("Failed to send message (client is being " +
-                "concurrently closed).");
-
-        try {
-            if (stopping)
-                throw new IgfsHadoopCommunicationException("Failed to send message (client is being concurrently closed).");
-
-            assert msg.command() == IgfsIpcCommand.WRITE_BLOCK;
-
-            IgfsStreamControlRequest req = (IgfsStreamControlRequest)msg;
-
-            byte[] hdr = IgfsMarshaller.createHeader(-1, IgfsIpcCommand.WRITE_BLOCK);
-
-            U.longToBytes(req.streamId(), hdr, 12);
-            U.intToBytes(req.length(), hdr, 20);
-
-            synchronized (this) {
-                out.write(hdr);
-                out.write(req.data(), (int)req.position(), req.length());
-
-                out.flush();
-            }
-        }
-        catch (IOException e) {
-            throw new IgfsHadoopCommunicationException(e);
-        }
-        finally {
-            busyLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * Closes client but does not wait.
-     *
-     * @param err Error.
-     */
-    private void close0(@Nullable Throwable err) {
-        busyLock.writeLock().lock();
-
-        try {
-            if (stopping)
-                return;
-
-            stopping = true;
-        }
-        finally {
-            busyLock.writeLock().unlock();
-        }
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to perform request (connection was concurrently closed before response " +
-                "is received).");
-
-        // Clean up resources.
-        U.closeQuiet(out);
-
-        if (endpoint != null)
-            endpoint.close();
-
-        // Unwind futures. We can safely iterate here because no more futures will be added.
-        Iterator<IgfsHadoopFuture> it = reqMap.values().iterator();
-
-        while (it.hasNext()) {
-            IgfsHadoopFuture fut = it.next();
-
-            fut.onDone(err);
-
-            it.remove();
-        }
-
-        for (IgfsHadoopIpcIoListener lsnr : lsnrs)
-            lsnr.onClose();
-    }
-
-    /**
-     * Do not extend {@code GridThread} to minimize class dependencies.
-     */
-    private class ReaderThread extends Thread {
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public void run() {
-            // Error to fail pending futures.
-            Throwable err = null;
-
-            try {
-                InputStream in = endpoint.inputStream();
-
-                IgfsDataInputStream dis = new IgfsDataInputStream(in);
-
-                byte[] hdr = new byte[IgfsMarshaller.HEADER_SIZE];
-                byte[] msgHdr = new byte[IgfsControlResponse.RES_HEADER_SIZE];
-
-                while (!Thread.currentThread().isInterrupted()) {
-                    dis.readFully(hdr);
-
-                    long reqId = U.bytesToLong(hdr, 0);
-
-                    // We don't wait for write responses, therefore reqId is -1.
-                    if (reqId == -1) {
-                        // We received a response which normally should not be sent. It must contain an error.
-                        dis.readFully(msgHdr);
-
-                        assert msgHdr[4] != 0;
-
-                        String errMsg = dis.readUTF();
-
-                        // Error code.
-                        dis.readInt();
-
-                        long streamId = dis.readLong();
-
-                        for (IgfsHadoopIpcIoListener lsnr : lsnrs)
-                            lsnr.onError(streamId, errMsg);
-                    }
-                    else {
-                        IgfsHadoopFuture<Object> fut = reqMap.remove(reqId);
-
-                        if (fut == null) {
-                            String msg = "Failed to read response from server: response closure is unavailable for " +
-                                "requestId (will close connection):" + reqId;
-
-                            log.warn(msg);
-
-                            err = new IgniteCheckedException(msg);
-
-                            break;
-                        }
-                        else {
-                            try {
-                                IgfsIpcCommand cmd = IgfsIpcCommand.valueOf(U.bytesToInt(hdr, 8));
-
-                                if (log.isDebugEnabled())
-                                    log.debug("Received IGFS response [reqId=" + reqId + ", cmd=" + cmd + ']');
-
-                                Object res = null;
-
-                                if (fut.read()) {
-                                    dis.readFully(msgHdr);
-
-                                    boolean hasErr = msgHdr[4] != 0;
-
-                                    if (hasErr) {
-                                        String errMsg = dis.readUTF();
-
-                                        // Error code.
-                                        Integer errCode = dis.readInt();
-
-                                        IgfsControlResponse.throwError(errCode, errMsg);
-                                    }
-
-                                    int blockLen = U.bytesToInt(msgHdr, 5);
-
-                                    int readLen = Math.min(blockLen, fut.outputLength());
-
-                                    if (readLen > 0) {
-                                        assert fut.outputBuffer() != null;
-
-                                        dis.readFully(fut.outputBuffer(), fut.outputOffset(), readLen);
-                                    }
-
-                                    if (readLen != blockLen) {
-                                        byte[] buf = new byte[blockLen - readLen];
-
-                                        dis.readFully(buf);
-
-                                        res = buf;
-                                    }
-                                }
-                                else
-                                    res = marsh.unmarshall(cmd, hdr, dis);
-
-                                fut.onDone(res);
-                            }
-                            catch (IgniteCheckedException e) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to apply response closure (will fail request future): " +
-                                        e.getMessage());
-
-                                fut.onDone(e);
-
-                                err = e;
-                            }
-                        }
-                    }
-                }
-            }
-            catch (EOFException ignored) {
-                err = new IgniteCheckedException("Failed to read response from server (connection was closed by remote peer).");
-            }
-            catch (IOException e) {
-                if (!stopping)
-                    log.error("Failed to read data (connection will be closed)", e);
-
-                err = new IgfsHadoopCommunicationException(e);
-            }
-            catch (IgniteCheckedException e) {
-                if (!stopping)
-                    log.error("Failed to obtain endpoint input stream (connection will be closed)", e);
-
-                err = e;
-            }
-            finally {
-                close0(err);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return getClass().getSimpleName() + " [endpointAddr=" + endpointAddr + ", activeCnt=" + activeCnt +
-            ", stopping=" + stopping + ']';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIoListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIoListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIoListener.java
deleted file mode 100644
index ffc58ba..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIpcIoListener.java
+++ /dev/null
@@ -1,36 +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.igfs.hadoop;
-
-/**
- * Listens to the events of {@link IgfsHadoopIpcIo}.
- */
-public interface IgfsHadoopIpcIoListener {
-    /**
-     * Callback invoked when the IO is being closed.
-     */
-    public void onClose();
-
-    /**
-     * Callback invoked when remote error occurs.
-     *
-     * @param streamId Stream ID.
-     * @param errMsg Error message.
-     */
-    public void onError(long streamId, String errMsg);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopJclLogger.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopJclLogger.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopJclLogger.java
deleted file mode 100644
index e43d77a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopJclLogger.java
+++ /dev/null
@@ -1,112 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.jetbrains.annotations.*;
-
-/**
- * JCL logger wrapper for Hadoop.
- */
-public class IgfsHadoopJclLogger implements IgniteLogger {
-    /** JCL implementation proxy. */
-    private Log impl;
-
-    /**
-     * Constructor.
-     *
-     * @param impl JCL implementation to use.
-     */
-    IgfsHadoopJclLogger(Log impl) {
-        assert impl != null;
-
-        this.impl = impl;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteLogger getLogger(Object ctgr) {
-        return new IgfsHadoopJclLogger(LogFactory.getLog(
-            ctgr instanceof Class ? ((Class)ctgr).getName() : String.valueOf(ctgr)));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void trace(String msg) {
-        impl.trace(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void debug(String msg) {
-        impl.debug(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void info(String msg) {
-        impl.info(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void warning(String msg) {
-        impl.warn(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void warning(String msg, @Nullable Throwable e) {
-        impl.warn(msg, e);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void error(String msg) {
-        impl.error(msg);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isQuiet() {
-        return !isInfoEnabled() && !isDebugEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void error(String msg, @Nullable Throwable e) {
-        impl.error(msg, e);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isTraceEnabled() {
-        return impl.isTraceEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isDebugEnabled() {
-        return impl.isDebugEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isInfoEnabled() {
-        return impl.isInfoEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public String fileName() {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return "IgfsHadoopJclLogger [impl=" + impl + ']';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutProc.java
deleted file mode 100644
index 31183a8..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutProc.java
+++ /dev/null
@@ -1,466 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.*;
-
-/**
- * Communication with external process (TCP or shmem).
- */
-public class IgfsHadoopOutProc implements IgfsHadoopEx, IgfsHadoopIpcIoListener {
-    /** Expected result is boolean. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Boolean> BOOL_RES = createClosure();
-
-    /** Expected result is boolean. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Long> LONG_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsFile> FILE_RES = createClosure();
-
-    /** Expected result is {@code IgfsHandshakeResponse} */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        IgfsHandshakeResponse> HANDSHAKE_RES = createClosure();
-
-    /** Expected result is {@code IgfsStatus} */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsStatus> STATUS_RES =
-        createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        IgfsInputStreamDescriptor> STREAM_DESCRIPTOR_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsFile>> FILE_COL_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsPath>> PATH_COL_RES = createClosure();
-
-    /** Expected result is {@code IgfsPathSummary}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsPathSummary> SUMMARY_RES =
-        createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsBlockLocation>> BLOCK_LOCATION_COL_RES = createClosure();
-
-    /** Grid name. */
-    private final String grid;
-
-    /** IGFS name. */
-    private final String igfs;
-
-    /** Client log. */
-    private final Log log;
-
-    /** Client IO. */
-    private final IgfsHadoopIpcIo io;
-
-    /** Event listeners. */
-    private final Map<Long, IgfsHadoopStreamEventListener> lsnrs = new ConcurrentHashMap8<>();
-
-    /**
-     * Constructor for TCP endpoint.
-     *
-     * @param host Host.
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    public IgfsHadoopOutProc(String host, int port, String grid, String igfs, Log log) throws IOException {
-        this(host, port, grid, igfs, false, log);
-    }
-
-    /**
-     * Constructor for shmem endpoint.
-     *
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    public IgfsHadoopOutProc(int port, String grid, String igfs, Log log) throws IOException {
-        this(null, port, grid, igfs, true, log);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param host Host.
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param shmem Shared memory flag.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    private IgfsHadoopOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log)
-        throws IOException {
-        assert host != null && !shmem || host == null && shmem :
-            "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
-
-        String endpoint = host != null ? host + ":" + port : "shmem:" + port;
-
-        this.grid = grid;
-        this.igfs = igfs;
-        this.log = log;
-
-        io = IgfsHadoopIpcIo.get(log, endpoint);
-
-        io.addEventListener(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException {
-        final IgfsHandshakeRequest req = new IgfsHandshakeRequest();
-
-        req.gridName(grid);
-        req.igfsName(igfs);
-        req.logDirectory(logDir);
-
-        return io.send(req).chain(HANDSHAKE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close(boolean force) {
-        assert io != null;
-
-        io.removeEventListener(this);
-
-        if (force)
-            io.forceClose();
-        else
-            io.release();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(INFO);
-        msg.path(path);
-
-        return io.send(msg).chain(FILE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(UPDATE);
-        msg.path(path);
-        msg.properties(props);
-
-        return io.send(msg).chain(FILE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(SET_TIMES);
-        msg.path(path);
-        msg.accessTime(accessTime);
-        msg.modificationTime(modificationTime);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(RENAME);
-        msg.path(src);
-        msg.destinationPath(dest);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(DELETE);
-        msg.path(path);
-        msg.flag(recursive);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
-        throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(AFFINITY);
-        msg.path(path);
-        msg.start(start);
-        msg.length(len);
-
-        return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(PATH_SUMMARY);
-        msg.path(path);
-
-        return io.send(msg).chain(SUMMARY_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(MAKE_DIRECTORIES);
-        msg.path(path);
-        msg.properties(props);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(LIST_FILES);
-        msg.path(path);
-
-        return io.send(msg).chain(FILE_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(LIST_PATHS);
-        msg.path(path);
-
-        return io.send(msg).chain(PATH_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
-        return io.send(new IgfsStatusRequest()).chain(STATUS_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_READ);
-        msg.path(path);
-        msg.flag(false);
-
-        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
-
-        return new IgfsHadoopStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(IgfsPath path,
-        int seqReadsBeforePrefetch) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_READ);
-        msg.path(path);
-        msg.flag(true);
-        msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
-
-        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
-
-        return new IgfsHadoopStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_CREATE);
-        msg.path(path);
-        msg.flag(overwrite);
-        msg.colocate(colocate);
-        msg.properties(props);
-        msg.replication(replication);
-        msg.blockSize(blockSize);
-
-        Long streamId = io.send(msg).chain(LONG_RES).get();
-
-        return new IgfsHadoopStreamDelegate(this, streamId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_APPEND);
-        msg.path(path);
-        msg.flag(create);
-        msg.properties(props);
-
-        Long streamId = io.send(msg).chain(LONG_RES).get();
-
-        return new IgfsHadoopStreamDelegate(this, streamId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridPlainFuture<byte[]> readData(IgfsHadoopStreamDelegate desc, long pos, int len,
-        final @Nullable byte[] outBuf, final int outOff, final int outLen) {
-        assert len > 0;
-
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(READ_BLOCK);
-        msg.streamId((long) desc.target());
-        msg.position(pos);
-        msg.length(len);
-
-        try {
-            return io.send(msg, outBuf, outOff, outLen);
-        }
-        catch (IgniteCheckedException e) {
-            return new GridPlainFutureAdapter<>(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeData(IgfsHadoopStreamDelegate desc, byte[] data, int off, int len)
-        throws IOException {
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(WRITE_BLOCK);
-        msg.streamId((long) desc.target());
-        msg.data(data);
-        msg.position(off);
-        msg.length(len);
-
-        try {
-            io.sendPlain(msg);
-        }
-        catch (IgniteCheckedException e) {
-            throw IgfsHadoopUtils.cast(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush(IgfsHadoopStreamDelegate delegate) throws IOException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void closeStream(IgfsHadoopStreamDelegate desc) throws IOException {
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(CLOSE);
-        msg.streamId((long)desc.target());
-
-        try {
-            io.send(msg).chain(BOOL_RES).get();
-        }
-        catch (IgniteCheckedException e) {
-            throw IgfsHadoopUtils.cast(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addEventListener(IgfsHadoopStreamDelegate desc,
-        IgfsHadoopStreamEventListener lsnr) {
-        long streamId = desc.target();
-
-        IgfsHadoopStreamEventListener lsnr0 = lsnrs.put(streamId, lsnr);
-
-        assert lsnr0 == null || lsnr0 == lsnr;
-
-        if (log.isDebugEnabled())
-            log.debug("Added stream event listener [streamId=" + streamId + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeEventListener(IgfsHadoopStreamDelegate desc) {
-        long streamId = desc.target();
-
-        IgfsHadoopStreamEventListener lsnr0 = lsnrs.remove(streamId);
-
-        if (lsnr0 != null && log.isDebugEnabled())
-            log.debug("Removed stream event listener [streamId=" + streamId + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() {
-        for (IgfsHadoopStreamEventListener lsnr : lsnrs.values()) {
-            try {
-                lsnr.onClose();
-            }
-            catch (IgniteCheckedException e) {
-                log.warn("Got exception from stream event listener (will ignore): " + lsnr, e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(long streamId, String errMsg) {
-        IgfsHadoopStreamEventListener lsnr = lsnrs.get(streamId);
-
-        if (lsnr != null)
-            lsnr.onError(errMsg);
-        else
-            log.warn("Received write error response for not registered output stream (will ignore) " +
-                "[streamId= " + streamId + ']');
-    }
-
-    /**
-     * Creates conversion closure for given type.
-     *
-     * @param <T> Type of expected result.
-     * @return Conversion closure.
-     */
-    @SuppressWarnings("unchecked")
-    private static <T> GridPlainClosure<GridPlainFuture<IgfsMessage>, T> createClosure() {
-        return new GridPlainClosure<GridPlainFuture<IgfsMessage>, T>() {
-            @Override public T apply(GridPlainFuture<IgfsMessage> fut) throws IgniteCheckedException {
-                IgfsControlResponse res = (IgfsControlResponse)fut.get();
-
-                if (res.hasError())
-                    res.throwError();
-
-                return (T)res.response();
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutputStream.java
deleted file mode 100644
index ae5f980..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopOutputStream.java
+++ /dev/null
@@ -1,201 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * IGFS Hadoop output stream implementation.
- */
-public class IgfsHadoopOutputStream extends OutputStream implements IgfsHadoopStreamEventListener {
-    /** Log instance. */
-    private Log log;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Log stream ID. */
-    private long logStreamId;
-
-    /** Server stream delegate. */
-    private IgfsHadoopStreamDelegate delegate;
-
-    /** Closed flag. */
-    private volatile boolean closed;
-
-    /** Flag set if stream was closed due to connection breakage. */
-    private boolean connBroken;
-
-    /** Error message. */
-    private volatile String errMsg;
-
-    /** Read time. */
-    private long writeTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of written bytes. */
-    private long total;
-
-    /**
-     * Creates light output stream.
-     *
-     * @param delegate Server stream delegate.
-     * @param log Logger to use.
-     * @param clientLog Client logger.
-     */
-    public IgfsHadoopOutputStream(IgfsHadoopStreamDelegate delegate, Log log,
-        IgfsLogger clientLog, long logStreamId) {
-        this.delegate = delegate;
-        this.log = log;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        lastTs = System.nanoTime();
-
-        delegate.hadoop().addEventListener(delegate, this);
-    }
-
-    /**
-     * Read start.
-     */
-    private void writeStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void writeEnd() {
-        long now = System.nanoTime();
-
-        writeTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
-        check();
-
-        writeStart();
-
-        try {
-            delegate.hadoop().writeData(delegate, b, off, len);
-
-            total += len;
-        }
-        finally {
-            writeEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(int b) throws IOException {
-        write(new byte[] {(byte)b});
-
-        total++;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush() throws IOException {
-        delegate.hadoop().flush(delegate);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        if (!closed) {
-            if (log.isDebugEnabled())
-                log.debug("Closing output stream: " + delegate);
-
-            writeStart();
-
-            delegate.hadoop().closeStream(delegate);
-
-            markClosed(false);
-
-            writeEnd();
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
-
-            if (log.isDebugEnabled())
-                log.debug("Closed output stream [delegate=" + delegate + ", writeTime=" + writeTime / 1000 +
-                    ", userTime=" + userTime / 1000 + ']');
-        }
-        else if(connBroken)
-            throw new IOException(
-                "Failed to close stream, because connection was broken (data could have been lost).");
-    }
-
-    /**
-     * Marks stream as closed.
-     *
-     * @param connBroken {@code True} if connection with server was lost.
-     */
-    private void markClosed(boolean connBroken) {
-        // It is ok to have race here.
-        if (!closed) {
-            closed = true;
-
-            delegate.hadoop().removeEventListener(delegate);
-
-            this.connBroken = connBroken;
-        }
-    }
-
-    /**
-     * @throws IOException If check failed.
-     */
-    private void check() throws IOException {
-        String errMsg0 = errMsg;
-
-        if (errMsg0 != null)
-            throw new IOException(errMsg0);
-
-        if (closed) {
-            if (connBroken)
-                throw new IOException("Server connection was lost.");
-            else
-                throw new IOException("Stream is closed.");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() throws IgniteCheckedException {
-        markClosed(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(String errMsg) {
-        this.errMsg = errMsg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyInputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyInputStream.java
deleted file mode 100644
index 330537d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyInputStream.java
+++ /dev/null
@@ -1,335 +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.igfs.hadoop;
-
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.internal.igfs.common.*;
-
-import java.io.*;
-
-/**
- * Secondary Hadoop file system input stream wrapper.
- */
-public class IgfsHadoopProxyInputStream extends InputStream implements Seekable, PositionedReadable {
-    /** Actual input stream to the secondary file system. */
-    private final FSDataInputStream is;
-
-    /** Client logger. */
-    private final IgfsLogger clientLog;
-
-    /** Log stream ID. */
-    private final long logStreamId;
-
-    /** Read time. */
-    private long readTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of read bytes. */
-    private long total;
-
-    /** Closed flag. */
-    private boolean closed;
-
-    /**
-     * Constructor.
-     *
-     * @param is Actual input stream to the secondary file system.
-     * @param clientLog Client log.
-     */
-    public IgfsHadoopProxyInputStream(FSDataInputStream is, IgfsLogger clientLog, long logStreamId) {
-        assert is != null;
-        assert clientLog != null;
-
-        this.is = is;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        lastTs = System.nanoTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(byte[] b) throws IOException {
-        readStart();
-
-        int res;
-
-        try {
-            res = is.read(b);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (res != -1)
-            total += res;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(byte[] b, int off, int len) throws IOException {
-        readStart();
-
-        int res;
-
-        try {
-            res = super.read(b, off, len);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (res != -1)
-            total += res;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long skip(long n) throws IOException {
-        readStart();
-
-        long res;
-
-        try {
-            res =  is.skip(n);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSkip(logStreamId, res);
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int available() throws IOException {
-        readStart();
-
-        try {
-            return is.available();
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void close() throws IOException {
-        if (!closed) {
-            closed = true;
-
-            readStart();
-
-            try {
-                is.close();
-            }
-            finally {
-                readEnd();
-            }
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void mark(int readLimit) {
-        readStart();
-
-        try {
-            is.mark(readLimit);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (clientLog.isLogEnabled())
-            clientLog.logMark(logStreamId, readLimit);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void reset() throws IOException {
-        readStart();
-
-        try {
-            is.reset();
-        }
-        finally {
-            readEnd();
-        }
-
-        if (clientLog.isLogEnabled())
-            clientLog.logReset(logStreamId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized boolean markSupported() {
-        readStart();
-
-        try {
-            return is.markSupported();
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read() throws IOException {
-        readStart();
-
-        int res;
-
-        try {
-            res = is.read();
-        }
-        finally {
-            readEnd();
-        }
-
-        if (res != -1)
-            total++;
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(long pos, byte[] buf, int off, int len) throws IOException {
-        readStart();
-
-        int res;
-
-        try {
-            res = is.read(pos, buf, off, len);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (res != -1)
-            total += res;
-
-        if (clientLog.isLogEnabled())
-            clientLog.logRandomRead(logStreamId, pos, res);
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void readFully(long pos, byte[] buf, int off, int len) throws IOException {
-        readStart();
-
-        try {
-            is.readFully(pos, buf, off, len);
-        }
-        finally {
-            readEnd();
-        }
-
-        total += len;
-
-        if (clientLog.isLogEnabled())
-            clientLog.logRandomRead(logStreamId, pos, len);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void readFully(long pos, byte[] buf) throws IOException {
-        readStart();
-
-        try {
-            is.readFully(pos, buf);
-        }
-        finally {
-            readEnd();
-        }
-
-        total += buf.length;
-
-        if (clientLog.isLogEnabled())
-            clientLog.logRandomRead(logStreamId, pos, buf.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void seek(long pos) throws IOException {
-        readStart();
-
-        try {
-            is.seek(pos);
-        }
-        finally {
-            readEnd();
-        }
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSeek(logStreamId, pos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long getPos() throws IOException {
-        readStart();
-
-        try {
-            return is.getPos();
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized boolean seekToNewSource(long targetPos) throws IOException {
-        readStart();
-
-        try {
-            return is.seekToNewSource(targetPos);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /**
-     * Read start.
-     */
-    private void readStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void readEnd() {
-        long now = System.nanoTime();
-
-        readTime += now - lastTs;
-
-        lastTs = now;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyOutputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyOutputStream.java
deleted file mode 100644
index 41e80eb..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopProxyOutputStream.java
+++ /dev/null
@@ -1,165 +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.igfs.hadoop;
-
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.internal.igfs.common.*;
-
-import java.io.*;
-
-/**
- * Secondary Hadoop file system output stream wrapper.
- */
-public class IgfsHadoopProxyOutputStream extends OutputStream {
-    /** Actual output stream. */
-    private FSDataOutputStream os;
-
-    /** Client logger. */
-    private final IgfsLogger clientLog;
-
-    /** Log stream ID. */
-    private final long logStreamId;
-
-    /** Read time. */
-    private long writeTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of written bytes. */
-    private long total;
-
-    /** Closed flag. */
-    private boolean closed;
-
-    /**
-     * Constructor.
-     *
-     * @param os Actual output stream.
-     * @param clientLog Client logger.
-     * @param logStreamId Log stream ID.
-     */
-    public IgfsHadoopProxyOutputStream(FSDataOutputStream os, IgfsLogger clientLog, long logStreamId) {
-        assert os != null;
-        assert clientLog != null;
-
-        this.os = os;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        lastTs = System.nanoTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void write(int b) throws IOException {
-        writeStart();
-
-        try {
-            os.write(b);
-        }
-        finally {
-            writeEnd();
-        }
-
-        total++;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void write(byte[] b) throws IOException {
-        writeStart();
-
-        try {
-            os.write(b);
-        }
-        finally {
-            writeEnd();
-        }
-
-        total += b.length;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void write(byte[] b, int off, int len) throws IOException {
-        writeStart();
-
-        try {
-            os.write(b, off, len);
-        }
-        finally {
-            writeEnd();
-        }
-
-        total += len;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void flush() throws IOException {
-        writeStart();
-
-        try {
-            os.flush();
-        }
-        finally {
-            writeEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void close() throws IOException {
-        if (!closed) {
-            closed = true;
-
-            writeStart();
-
-            try {
-                os.close();
-            }
-            finally {
-                writeEnd();
-            }
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
-        }
-    }
-
-    /**
-     * Read start.
-     */
-    private void writeStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void writeEnd() {
-        long now = System.nanoTime();
-
-        writeTime += now - lastTs;
-
-        lastTs = now;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopReader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopReader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopReader.java
deleted file mode 100644
index 3ab3acc..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopReader.java
+++ /dev/null
@@ -1,104 +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.igfs.hadoop;
-
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Secondary file system input stream wrapper which actually opens input stream only in case it is explicitly
- * requested.
- * <p>
- * The class is expected to be used only from synchronized context and therefore is not tread-safe.
- */
-public class IgfsHadoopReader implements IgfsReader {
-    /** Secondary file system. */
-    private final FileSystem fs;
-
-    /** Path to the file to open. */
-    private final Path path;
-
-    /** Buffer size. */
-    private final int bufSize;
-
-    /** Actual input stream. */
-    private FSDataInputStream in;
-
-    /** Cached error occurred during output stream open. */
-    private IOException err;
-
-    /** Flag indicating that the stream was already opened. */
-    private boolean opened;
-
-    /**
-     * Constructor.
-     *
-     * @param fs Secondary file system.
-     * @param path Path to the file to open.
-     * @param bufSize Buffer size.
-     */
-    public IgfsHadoopReader(FileSystem fs, Path path, int bufSize) {
-        assert fs != null;
-        assert path != null;
-
-        this.fs = fs;
-        this.path = path;
-        this.bufSize = bufSize;
-    }
-
-    /** Get input stream. */
-    private PositionedReadable in() throws IOException {
-        if (opened) {
-            if (err != null)
-                throw err;
-        }
-        else {
-            opened = true;
-
-            try {
-                in = fs.open(path, bufSize);
-
-                if (in == null)
-                    throw new IOException("Failed to open input stream (file system returned null): " + path);
-            }
-            catch (IOException e) {
-                err = e;
-
-                throw err;
-            }
-        }
-
-        return in;
-    }
-
-    /**
-     * Close wrapped input stream in case it was previously opened.
-     */
-    @Override public void close() {
-        U.closeQuiet(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int read(long pos, byte[] buf, int off, int len) throws IOException {
-        return in().read(pos, buf, off, len);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamDelegate.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamDelegate.java
deleted file mode 100644
index 9aaab4c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamDelegate.java
+++ /dev/null
@@ -1,96 +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.igfs.hadoop;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-/**
- * IGFS Hadoop stream descriptor.
- */
-public class IgfsHadoopStreamDelegate {
-    /** RPC handler. */
-    private final IgfsHadoopEx hadoop;
-
-    /** Target. */
-    private final Object target;
-
-    /** Optional stream length. */
-    private final long len;
-
-    /**
-     * Constructor.
-     *
-     * @param target Target.
-     */
-    public IgfsHadoopStreamDelegate(IgfsHadoopEx hadoop, Object target) {
-        this(hadoop, target, -1);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param target Target.
-     * @param len Optional length.
-     */
-    public IgfsHadoopStreamDelegate(IgfsHadoopEx hadoop, Object target, long len) {
-        assert hadoop != null;
-        assert target != null;
-
-        this.hadoop = hadoop;
-        this.target = target;
-        this.len = len;
-    }
-
-    /**
-     * @return RPC handler.
-     */
-    public IgfsHadoopEx hadoop() {
-        return hadoop;
-    }
-
-    /**
-     * @return Stream target.
-     */
-    @SuppressWarnings("unchecked")
-    public <T> T target() {
-        return (T) target;
-    }
-
-    /**
-     * @return Length.
-     */
-    public long length() {
-        return len;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return System.identityHashCode(target);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        return obj != null && obj instanceof IgfsHadoopStreamDelegate &&
-            target == ((IgfsHadoopStreamDelegate)obj).target;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsHadoopStreamDelegate.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamEventListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamEventListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamEventListener.java
deleted file mode 100644
index 20d7f2a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopStreamEventListener.java
+++ /dev/null
@@ -1,39 +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.igfs.hadoop;
-
-import org.apache.ignite.*;
-
-/**
- * IGFS input stream event listener.
- */
-public interface IgfsHadoopStreamEventListener {
-    /**
-     * Callback invoked when the stream is being closed.
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    public void onClose() throws IgniteCheckedException;
-
-    /**
-     * Callback invoked when remote error occurs.
-     *
-     * @param errMsg Error message.
-     */
-    public void onError(String errMsg);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopUtils.java
deleted file mode 100644
index bd96e60..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopUtils.java
+++ /dev/null
@@ -1,131 +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.igfs.hadoop;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Utility constants and methods for IGFS Hadoop file system.
- */
-public class IgfsHadoopUtils {
-    /** Parameter name for endpoint no embed mode flag. */
-    public static final String PARAM_IGFS_ENDPOINT_NO_EMBED = "fs.igfs.%s.endpoint.no_embed";
-
-    /** Parameter name for endpoint no shared memory flag. */
-    public static final String PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM = "fs.igfs.%s.endpoint.no_local_shmem";
-
-    /** Parameter name for endpoint no local TCP flag. */
-    public static final String PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP = "fs.igfs.%s.endpoint.no_local_tcp";
-
-    /**
-     * Get string parameter.
-     *
-     * @param cfg Configuration.
-     * @param name Parameter name.
-     * @param authority Authority.
-     * @param dflt Default value.
-     * @return String value.
-     */
-    public static String parameter(Configuration cfg, String name, String authority, String dflt) {
-        return cfg.get(String.format(name, authority != null ? authority : ""), dflt);
-    }
-
-    /**
-     * Get integer parameter.
-     *
-     * @param cfg Configuration.
-     * @param name Parameter name.
-     * @param authority Authority.
-     * @param dflt Default value.
-     * @return Integer value.
-     * @throws IOException In case of parse exception.
-     */
-    public static int parameter(Configuration cfg, String name, String authority, int dflt) throws IOException {
-        String name0 = String.format(name, authority != null ? authority : "");
-
-        try {
-            return cfg.getInt(name0, dflt);
-        }
-        catch (NumberFormatException ignore) {
-            throw new IOException("Failed to parse parameter value to integer: " + name0);
-        }
-    }
-
-    /**
-     * Get boolean parameter.
-     *
-     * @param cfg Configuration.
-     * @param name Parameter name.
-     * @param authority Authority.
-     * @param dflt Default value.
-     * @return Boolean value.
-     */
-    public static boolean parameter(Configuration cfg, String name, String authority, boolean dflt) {
-        return cfg.getBoolean(String.format(name, authority != null ? authority : ""), dflt);
-    }
-
-    /**
-     * Cast Ignite exception to appropriate IO exception.
-     *
-     * @param e Exception to cast.
-     * @return Casted exception.
-     */
-    public static IOException cast(IgniteCheckedException e) {
-        return cast(e, null);
-    }
-
-    /**
-     * Cast Ignite exception to appropriate IO exception.
-     *
-     * @param e Exception to cast.
-     * @param path Path for exceptions.
-     * @return Casted exception.
-     */
-    @SuppressWarnings("unchecked")
-    public static IOException cast(IgniteCheckedException e, @Nullable String path) {
-        assert e != null;
-
-        // First check for any nested IOException; if exists - re-throw it.
-        if (e.hasCause(IOException.class))
-            return e.getCause(IOException.class);
-        else if (e.hasCause(IgfsFileNotFoundException.class))
-            return new FileNotFoundException(path); // TODO: Or PathNotFoundException?
-        else if (e.hasCause(IgfsParentNotDirectoryException.class))
-            return new ParentNotDirectoryException(path);
-        else if (path != null && e.hasCause(IgfsDirectoryNotEmptyException.class))
-            return new PathIsNotEmptyDirectoryException(path);
-        else if (path != null && e.hasCause(IgfsPathAlreadyExistsException.class))
-            return new PathExistsException(path);
-        else
-            return new IOException(e);
-    }
-
-    /**
-     * Constructor.
-     */
-    private IgfsHadoopUtils() {
-        // No-op.
-    }
-}


[28/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamDelegate.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamDelegate.java
new file mode 100644
index 0000000..e64b761
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamDelegate.java
@@ -0,0 +1,96 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+/**
+ * IGFS Hadoop stream descriptor.
+ */
+public class HadoopIgfsStreamDelegate {
+    /** RPC handler. */
+    private final HadoopIgfsEx hadoop;
+
+    /** Target. */
+    private final Object target;
+
+    /** Optional stream length. */
+    private final long len;
+
+    /**
+     * Constructor.
+     *
+     * @param target Target.
+     */
+    public HadoopIgfsStreamDelegate(HadoopIgfsEx hadoop, Object target) {
+        this(hadoop, target, -1);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param target Target.
+     * @param len Optional length.
+     */
+    public HadoopIgfsStreamDelegate(HadoopIgfsEx hadoop, Object target, long len) {
+        assert hadoop != null;
+        assert target != null;
+
+        this.hadoop = hadoop;
+        this.target = target;
+        this.len = len;
+    }
+
+    /**
+     * @return RPC handler.
+     */
+    public HadoopIgfsEx hadoop() {
+        return hadoop;
+    }
+
+    /**
+     * @return Stream target.
+     */
+    @SuppressWarnings("unchecked")
+    public <T> T target() {
+        return (T) target;
+    }
+
+    /**
+     * @return Length.
+     */
+    public long length() {
+        return len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return System.identityHashCode(target);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        return obj != null && obj instanceof HadoopIgfsStreamDelegate &&
+            target == ((HadoopIgfsStreamDelegate)obj).target;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopIgfsStreamDelegate.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamEventListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamEventListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamEventListener.java
new file mode 100644
index 0000000..b3de523
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsStreamEventListener.java
@@ -0,0 +1,39 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.ignite.*;
+
+/**
+ * IGFS input stream event listener.
+ */
+public interface HadoopIgfsStreamEventListener {
+    /**
+     * Callback invoked when the stream is being closed.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onClose() throws IgniteCheckedException;
+
+    /**
+     * Callback invoked when remote error occurs.
+     *
+     * @param errMsg Error message.
+     */
+    public void onError(String errMsg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsUtils.java
new file mode 100644
index 0000000..009443d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsUtils.java
@@ -0,0 +1,131 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Utility constants and methods for IGFS Hadoop file system.
+ */
+public class HadoopIgfsUtils {
+    /** Parameter name for endpoint no embed mode flag. */
+    public static final String PARAM_IGFS_ENDPOINT_NO_EMBED = "fs.igfs.%s.endpoint.no_embed";
+
+    /** Parameter name for endpoint no shared memory flag. */
+    public static final String PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM = "fs.igfs.%s.endpoint.no_local_shmem";
+
+    /** Parameter name for endpoint no local TCP flag. */
+    public static final String PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP = "fs.igfs.%s.endpoint.no_local_tcp";
+
+    /**
+     * Get string parameter.
+     *
+     * @param cfg Configuration.
+     * @param name Parameter name.
+     * @param authority Authority.
+     * @param dflt Default value.
+     * @return String value.
+     */
+    public static String parameter(Configuration cfg, String name, String authority, String dflt) {
+        return cfg.get(String.format(name, authority != null ? authority : ""), dflt);
+    }
+
+    /**
+     * Get integer parameter.
+     *
+     * @param cfg Configuration.
+     * @param name Parameter name.
+     * @param authority Authority.
+     * @param dflt Default value.
+     * @return Integer value.
+     * @throws IOException In case of parse exception.
+     */
+    public static int parameter(Configuration cfg, String name, String authority, int dflt) throws IOException {
+        String name0 = String.format(name, authority != null ? authority : "");
+
+        try {
+            return cfg.getInt(name0, dflt);
+        }
+        catch (NumberFormatException ignore) {
+            throw new IOException("Failed to parse parameter value to integer: " + name0);
+        }
+    }
+
+    /**
+     * Get boolean parameter.
+     *
+     * @param cfg Configuration.
+     * @param name Parameter name.
+     * @param authority Authority.
+     * @param dflt Default value.
+     * @return Boolean value.
+     */
+    public static boolean parameter(Configuration cfg, String name, String authority, boolean dflt) {
+        return cfg.getBoolean(String.format(name, authority != null ? authority : ""), dflt);
+    }
+
+    /**
+     * Cast Ignite exception to appropriate IO exception.
+     *
+     * @param e Exception to cast.
+     * @return Casted exception.
+     */
+    public static IOException cast(IgniteCheckedException e) {
+        return cast(e, null);
+    }
+
+    /**
+     * Cast Ignite exception to appropriate IO exception.
+     *
+     * @param e Exception to cast.
+     * @param path Path for exceptions.
+     * @return Casted exception.
+     */
+    @SuppressWarnings("unchecked")
+    public static IOException cast(IgniteCheckedException e, @Nullable String path) {
+        assert e != null;
+
+        // First check for any nested IOException; if exists - re-throw it.
+        if (e.hasCause(IOException.class))
+            return e.getCause(IOException.class);
+        else if (e.hasCause(IgfsFileNotFoundException.class))
+            return new FileNotFoundException(path); // TODO: Or PathNotFoundException?
+        else if (e.hasCause(IgfsParentNotDirectoryException.class))
+            return new ParentNotDirectoryException(path);
+        else if (path != null && e.hasCause(IgfsDirectoryNotEmptyException.class))
+            return new PathIsNotEmptyDirectoryException(path);
+        else if (path != null && e.hasCause(IgfsPathAlreadyExistsException.class))
+            return new PathExistsException(path);
+        else
+            return new IOException(e);
+    }
+
+    /**
+     * Constructor.
+     */
+    private HadoopIgfsUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
new file mode 100644
index 0000000..94a4449
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIgfsWrapper.java
@@ -0,0 +1,511 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.hadoop.conf.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.igfs.hadoop.HadoopIgfsEndpoint.*;
+import static org.apache.ignite.internal.igfs.hadoop.HadoopIgfsUtils.*;
+
+/**
+ * Wrapper for IGFS server.
+ */
+public class HadoopIgfsWrapper implements HadoopIgfs {
+    /** Delegate. */
+    private final AtomicReference<Delegate> delegateRef = new AtomicReference<>();
+
+    /** Authority. */
+    private final String authority;
+
+    /** Connection string. */
+    private final HadoopIgfsEndpoint endpoint;
+
+    /** Log directory. */
+    private final String logDir;
+
+    /** Configuration. */
+    private final Configuration conf;
+
+    /** Logger. */
+    private final Log log;
+
+    /**
+     * Constructor.
+     *
+     * @param authority Authority (connection string).
+     * @param logDir Log directory for server.
+     * @param conf Configuration.
+     * @param log Current logger.
+     */
+    public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log) throws IOException {
+        try {
+            this.authority = authority;
+            this.endpoint = new HadoopIgfsEndpoint(authority);
+            this.logDir = logDir;
+            this.conf = conf;
+            this.log = log;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IOException("Failed to parse endpoint: " + authority, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsHandshakeResponse handshake(String logDir) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<IgfsHandshakeResponse>() {
+            @Override public IgfsHandshakeResponse apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) {
+                return hndResp;
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(boolean force) {
+        Delegate delegate = delegateRef.get();
+
+        if (delegate != null && delegateRef.compareAndSet(delegate, null))
+            delegate.close(force);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(final IgfsPath path) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<IgfsFile>() {
+            @Override public IgfsFile apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.info(path);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<IgfsFile>() {
+            @Override public IgfsFile apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.update(path, props);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime)
+        throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Boolean>() {
+            @Override public Boolean apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.setTimes(path, accessTime, modificationTime);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean rename(final IgfsPath src, final IgfsPath dest) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Boolean>() {
+            @Override public Boolean apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.rename(src, dest);
+            }
+        }, src);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean delete(final IgfsPath path, final boolean recursive) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Boolean>() {
+            @Override public Boolean apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.delete(path, recursive);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsBlockLocation> affinity(final IgfsPath path, final long start,
+        final long len) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Collection<IgfsBlockLocation>>() {
+            @Override public Collection<IgfsBlockLocation> apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.affinity(path, start, len);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsPathSummary contentSummary(final IgfsPath path) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<IgfsPathSummary>() {
+            @Override public IgfsPathSummary apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.contentSummary(path);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Boolean>() {
+            @Override public Boolean apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.mkdirs(path, props);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(final IgfsPath path) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Collection<IgfsFile>>() {
+            @Override public Collection<IgfsFile> apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.listFiles(path);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(final IgfsPath path) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<Collection<IgfsPath>>() {
+            @Override public Collection<IgfsPath> apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.listPaths(path);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsStatus fsStatus() throws IOException {
+        return withReconnectHandling(new FileSystemClosure<IgfsStatus>() {
+            @Override public IgfsStatus apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp)
+                throws IgniteCheckedException, IOException {
+                return hadoop.fsStatus();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<HadoopIgfsStreamDelegate>() {
+            @Override public HadoopIgfsStreamDelegate apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.open(path);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path, final int seqReadsBeforePrefetch)
+        throws IOException {
+        return withReconnectHandling(new FileSystemClosure<HadoopIgfsStreamDelegate>() {
+            @Override public HadoopIgfsStreamDelegate apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.open(path, seqReadsBeforePrefetch);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate create(final IgfsPath path, final boolean overwrite,
+        final boolean colocate, final int replication, final long blockSize, @Nullable final Map<String, String> props)
+        throws IOException {
+        return withReconnectHandling(new FileSystemClosure<HadoopIgfsStreamDelegate>() {
+            @Override public HadoopIgfsStreamDelegate apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.create(path, overwrite, colocate, replication, blockSize, props);
+            }
+        }, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate append(final IgfsPath path, final boolean create,
+        @Nullable final Map<String, String> props) throws IOException {
+        return withReconnectHandling(new FileSystemClosure<HadoopIgfsStreamDelegate>() {
+            @Override public HadoopIgfsStreamDelegate apply(HadoopIgfsEx hadoop,
+                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
+                return hadoop.append(path, create, props);
+            }
+        }, path);
+    }
+
+    /**
+     * Execute closure which is not path-specific.
+     *
+     * @param clo Closure.
+     * @return Result.
+     * @throws IOException If failed.
+     */
+    private <T> T withReconnectHandling(FileSystemClosure<T> clo) throws IOException {
+        return withReconnectHandling(clo, null);
+    }
+
+    /**
+     * Execute closure.
+     *
+     * @param clo Closure.
+     * @param path Path for exceptions.
+     * @return Result.
+     * @throws IOException If failed.
+     */
+    private <T> T withReconnectHandling(final FileSystemClosure<T> clo, @Nullable IgfsPath path)
+        throws IOException {
+        Exception err = null;
+
+        for (int i = 0; i < 2; i++) {
+            Delegate curDelegate = null;
+
+            boolean close = false;
+            boolean force = false;
+
+            try {
+                curDelegate = delegate();
+
+                assert curDelegate != null;
+
+                close = curDelegate.doomed;
+
+                return clo.apply(curDelegate.hadoop, curDelegate.hndResp);
+            }
+            catch (HadoopIgfsCommunicationException e) {
+                if (curDelegate != null && !curDelegate.doomed) {
+                    // Try getting rid fo faulty delegate ASAP.
+                    delegateRef.compareAndSet(curDelegate, null);
+
+                    close = true;
+                    force = true;
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send message to a server: " + e);
+
+                err = e;
+            }
+            catch (IgniteCheckedException e) {
+                throw HadoopIgfsUtils.cast(e, path != null ? path.toString() : null);
+            }
+            finally {
+                if (close) {
+                    assert curDelegate != null;
+
+                    curDelegate.close(force);
+                }
+            }
+        }
+
+        throw new IOException("Failed to communicate with IGFS.", err);
+    }
+
+    /**
+     * Get delegate creating it if needed.
+     *
+     * @return Delegate.
+     */
+    private Delegate delegate() throws HadoopIgfsCommunicationException {
+        Exception err = null;
+
+        // 1. If delegate is set, return it immediately.
+        Delegate curDelegate = delegateRef.get();
+
+        if (curDelegate != null)
+            return curDelegate;
+
+        // 2. Guess that we are in the same VM.
+        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false)) {
+            IgfsEx igfs = null;
+
+            if (endpoint.grid() == null) {
+                try {
+                    Ignite ignite = G.ignite();
+
+                    igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
+                }
+                catch (Exception e) {
+                    err = e;
+                }
+            }
+            else {
+                for (Ignite ignite : G.allGrids()) {
+                    try {
+                        igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
+
+                        break;
+                    }
+                    catch (Exception e) {
+                        err = e;
+                    }
+                }
+            }
+
+            if (igfs != null) {
+                HadoopIgfsEx hadoop = null;
+
+                try {
+                    hadoop = new HadoopIgfsInProc(igfs, log);
+
+                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+                }
+                catch (IOException | IgniteCheckedException e) {
+                    if (e instanceof HadoopIgfsCommunicationException)
+                        hadoop.close(true);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to connect to in-proc IGFS, fallback to IPC mode.", e);
+
+                    err = e;
+                }
+            }
+        }
+
+        // 3. Try connecting using shmem.
+        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority, false)) {
+            if (curDelegate == null && !U.isWindows()) {
+                HadoopIgfsEx hadoop = null;
+
+                try {
+                    hadoop = new HadoopOutProcIgfs(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+
+                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+                }
+                catch (IOException | IgniteCheckedException e) {
+                    if (e instanceof HadoopIgfsCommunicationException)
+                        hadoop.close(true);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to connect to out-proc local IGFS using shmem.", e);
+
+                    err = e;
+                }
+            }
+        }
+
+        // 4. Try local TCP connection.
+        boolean skipLocTcp = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP, authority, false);
+
+        if (!skipLocTcp) {
+            if (curDelegate == null) {
+                HadoopIgfsEx hadoop = null;
+
+                try {
+                    hadoop = new HadoopOutProcIgfs(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
+                        log);
+
+                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+                }
+                catch (IOException | IgniteCheckedException e) {
+                    if (e instanceof HadoopIgfsCommunicationException)
+                        hadoop.close(true);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Failed to connect to out-proc local IGFS using TCP.", e);
+
+                    err = e;
+                }
+            }
+        }
+
+        // 5. Try remote TCP connection.
+        if (curDelegate == null && (skipLocTcp || !F.eq(LOCALHOST, endpoint.host()))) {
+            HadoopIgfsEx hadoop = null;
+
+            try {
+                hadoop = new HadoopOutProcIgfs(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+
+                curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
+            }
+            catch (IOException | IgniteCheckedException e) {
+                if (e instanceof HadoopIgfsCommunicationException)
+                    hadoop.close(true);
+
+                if (log.isDebugEnabled())
+                    log.debug("Failed to connect to out-proc remote IGFS using TCP.", e);
+
+                err = e;
+            }
+        }
+
+        if (curDelegate != null) {
+            if (!delegateRef.compareAndSet(null, curDelegate))
+                curDelegate.doomed = true;
+
+            return curDelegate;
+        }
+        else
+            throw new HadoopIgfsCommunicationException("Failed to connect to IGFS: " + endpoint, err);
+    }
+
+    /**
+     * File system operation closure.
+     */
+    private static interface FileSystemClosure<T> {
+        /**
+         * Call closure body.
+         *
+         * @param hadoop RPC handler.
+         * @param hndResp Handshake response.
+         * @return Result.
+         * @throws IgniteCheckedException If failed.
+         * @throws IOException If failed.
+         */
+        public T apply(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException;
+    }
+
+    /**
+     * Delegate.
+     */
+    private static class Delegate {
+        /** RPC handler. */
+        private final HadoopIgfsEx hadoop;
+
+        /** Handshake request. */
+        private final IgfsHandshakeResponse hndResp;
+
+        /** Close guard. */
+        private final AtomicBoolean closeGuard = new AtomicBoolean();
+
+        /** Whether this delegate must be closed at the end of the next invocation. */
+        private boolean doomed;
+
+        /**
+         * Constructor.
+         *
+         * @param hadoop Hadoop.
+         * @param hndResp Handshake response.
+         */
+        private Delegate(HadoopIgfsEx hadoop, IgfsHandshakeResponse hndResp) {
+            this.hadoop = hadoop;
+            this.hndResp = hndResp;
+        }
+
+        /**
+         * Close underlying RPC handler.
+         *
+         * @param force Force flag.
+         */
+        private void close(boolean force) {
+            if (closeGuard.compareAndSet(false, true))
+                hadoop.close(force);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
new file mode 100644
index 0000000..5a008bd
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopInputIgfsStream.java
@@ -0,0 +1,626 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * IGFS input stream wrapper for hadoop interfaces.
+ */
+@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+public final class HadoopInputIgfsStream extends InputStream implements Seekable, PositionedReadable,
+    HadoopIgfsStreamEventListener {
+    /** Minimum buffer size. */
+    private static final int MIN_BUF_SIZE = 4 * 1024;
+
+    /** Server stream delegate. */
+    private HadoopIgfsStreamDelegate delegate;
+
+    /** Stream ID used by logger. */
+    private long logStreamId;
+
+    /** Stream position. */
+    private long pos;
+
+    /** Stream read limit. */
+    private long limit;
+
+    /** Mark position. */
+    private long markPos = -1;
+
+    /** Prefetch buffer. */
+    private DoubleFetchBuffer buf = new DoubleFetchBuffer();
+
+    /** Buffer half size for double-buffering. */
+    private int bufHalfSize;
+
+    /** Closed flag. */
+    private volatile boolean closed;
+
+    /** Flag set if stream was closed due to connection breakage. */
+    private boolean connBroken;
+
+    /** Logger. */
+    private Log log;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Read time. */
+    private long readTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of read bytes. */
+    private long total;
+
+    /**
+     * Creates input stream.
+     *
+     * @param delegate Server stream delegate.
+     * @param limit Read limit.
+     * @param bufSize Buffer size.
+     * @param log Log.
+     * @param clientLog Client logger.
+     */
+    public HadoopInputIgfsStream(HadoopIgfsStreamDelegate delegate, long limit, int bufSize, Log log,
+        IgfsLogger clientLog, long logStreamId) {
+        assert limit >= 0;
+
+        this.delegate = delegate;
+        this.limit = limit;
+        this.log = log;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        bufHalfSize = Math.max(bufSize, MIN_BUF_SIZE);
+
+        lastTs = System.nanoTime();
+
+        delegate.hadoop().addEventListener(delegate, this);
+    }
+
+    /**
+     * Read start.
+     */
+    private void readStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void readEnd() {
+        long now = System.nanoTime();
+
+        readTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read() throws IOException {
+        checkClosed();
+
+        readStart();
+
+        try {
+            if (eof())
+                return -1;
+
+            buf.refreshAhead(pos);
+
+            int res = buf.atPosition(pos);
+
+            pos++;
+            total++;
+
+            buf.refreshAhead(pos);
+
+            return res;
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(@NotNull byte[] b, int off, int len) throws IOException {
+        checkClosed();
+
+        if (eof())
+            return -1;
+
+        readStart();
+
+        try {
+            long remaining = limit - pos;
+
+            int read = buf.flatten(b, pos, off, len);
+
+            pos += read;
+            total += read;
+            remaining -= read;
+
+            if (remaining > 0 && read != len) {
+                int readAmt = (int)Math.min(remaining, len - read);
+
+                delegate.hadoop().readData(delegate, pos, readAmt, b, off + read, len - read).get();
+
+                read += readAmt;
+                pos += readAmt;
+                total += readAmt;
+            }
+
+            buf.refreshAhead(pos);
+
+            return read;
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long skip(long n) throws IOException {
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSkip(logStreamId, n);
+
+        long oldPos = pos;
+
+        if (pos + n <= limit)
+            pos += n;
+        else
+            pos = limit;
+
+        buf.refreshAhead(pos);
+
+        return pos - oldPos;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int available() throws IOException {
+        checkClosed();
+
+        int available = buf.available(pos);
+
+        assert available >= 0;
+
+        return available;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void close() throws IOException {
+        if (!closed) {
+            readStart();
+
+            if (log.isDebugEnabled())
+                log.debug("Closing input stream: " + delegate);
+
+            delegate.hadoop().closeStream(delegate);
+
+            readEnd();
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
+
+            markClosed(false);
+
+            if (log.isDebugEnabled())
+                log.debug("Closed stream [delegate=" + delegate + ", readTime=" + readTime +
+                    ", userTime=" + userTime + ']');
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void mark(int readLimit) {
+        markPos = pos;
+
+        if (clientLog.isLogEnabled())
+            clientLog.logMark(logStreamId, readLimit);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void reset() throws IOException {
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logReset(logStreamId);
+
+        if (markPos == -1)
+            throw new IOException("Stream was not marked.");
+
+        pos = markPos;
+
+        buf.refreshAhead(pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean markSupported() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized int read(long position, byte[] buf, int off, int len) throws IOException {
+        long remaining = limit - position;
+
+        int read = (int)Math.min(len, remaining);
+
+        // Return -1 at EOF.
+        if (read == 0)
+            return -1;
+
+        readFully(position, buf, off, read);
+
+        return read;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void readFully(long position, byte[] buf, int off, int len) throws IOException {
+        long remaining = limit - position;
+
+        checkClosed();
+
+        if (len > remaining)
+            throw new EOFException("End of stream reached before data was fully read.");
+
+        readStart();
+
+        try {
+            int read = this.buf.flatten(buf, position, off, len);
+
+            total += read;
+
+            if (read != len) {
+                int readAmt = len - read;
+
+                delegate.hadoop().readData(delegate, position + read, readAmt, buf, off + read, readAmt).get();
+
+                total += readAmt;
+            }
+
+            if (clientLog.isLogEnabled())
+                clientLog.logRandomRead(logStreamId, position, len);
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+        finally {
+            readEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(long position, byte[] buf) throws IOException {
+        readFully(position, buf, 0, buf.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized void seek(long pos) throws IOException {
+        A.ensure(pos >= 0, "position must be non-negative");
+
+        checkClosed();
+
+        if (clientLog.isLogEnabled())
+            clientLog.logSeek(logStreamId, pos);
+
+        if (pos > limit)
+            pos = limit;
+
+        if (log.isDebugEnabled())
+            log.debug("Seek to position [delegate=" + delegate + ", pos=" + pos + ", oldPos=" + this.pos + ']');
+
+        this.pos = pos;
+
+        buf.refreshAhead(pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized long getPos() {
+        return pos;
+    }
+
+    /** {@inheritDoc} */
+    @Override public synchronized boolean seekToNewSource(long targetPos) {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        markClosed(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(String errMsg) {
+        // No-op.
+    }
+
+    /**
+     * Marks stream as closed.
+     *
+     * @param connBroken {@code True} if connection with server was lost.
+     */
+    private void markClosed(boolean connBroken) {
+        // It is ok to have race here.
+        if (!closed) {
+            closed = true;
+
+            this.connBroken = connBroken;
+
+            delegate.hadoop().removeEventListener(delegate);
+        }
+    }
+
+    /**
+     * @throws IOException If check failed.
+     */
+    private void checkClosed() throws IOException {
+        if (closed) {
+            if (connBroken)
+                throw new IOException("Server connection was lost.");
+            else
+                throw new IOException("Stream is closed.");
+        }
+    }
+
+    /**
+     * @return {@code True} if end of stream reached.
+     */
+    private boolean eof() {
+        return limit == pos;
+    }
+
+    /**
+     * Asynchronous prefetch buffer.
+     */
+    private static class FetchBufferPart {
+        /** Read future. */
+        private GridPlainFuture<byte[]> readFut;
+
+        /** Position of cached chunk in file. */
+        private long pos;
+
+        /** Prefetch length. Need to store as read future result might be not available yet. */
+        private int len;
+
+        /**
+         * Creates fetch buffer part.
+         *
+         * @param readFut Read future for this buffer.
+         * @param pos Read position.
+         * @param len Chunk length.
+         */
+        private FetchBufferPart(GridPlainFuture<byte[]> readFut, long pos, int len) {
+            this.readFut = readFut;
+            this.pos = pos;
+            this.len = len;
+        }
+
+        /**
+         * Copies cached data if specified position matches cached region.
+         *
+         * @param dst Destination buffer.
+         * @param pos Read position in file.
+         * @param dstOff Offset in destination buffer from which start writing.
+         * @param len Maximum number of bytes to copy.
+         * @return Number of bytes copied.
+         * @throws IgniteCheckedException If read future failed.
+         */
+        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
+            // If read start position is within cached boundaries.
+            if (contains(pos)) {
+                byte[] data = readFut.get();
+
+                int srcPos = (int)(pos - this.pos);
+                int cpLen = Math.min(len, data.length - srcPos);
+
+                U.arrayCopy(data, srcPos, dst, dstOff, cpLen);
+
+                return cpLen;
+            }
+
+            return 0;
+        }
+
+        /**
+         * @return {@code True} if data is ready to be read.
+         */
+        public boolean ready() {
+            return readFut.isDone();
+        }
+
+        /**
+         * Checks if current buffer part contains given position.
+         *
+         * @param pos Position to check.
+         * @return {@code True} if position matches buffer region.
+         */
+        public boolean contains(long pos) {
+            return this.pos <= pos && this.pos + len > pos;
+        }
+    }
+
+    private class DoubleFetchBuffer {
+        /**  */
+        private FetchBufferPart first;
+
+        /** */
+        private FetchBufferPart second;
+
+        /**
+         * Copies fetched data from both buffers to destination array if cached region matched read position.
+         *
+         * @param dst Destination buffer.
+         * @param pos Read position in file.
+         * @param dstOff Destination buffer offset.
+         * @param len Maximum number of bytes to copy.
+         * @return Number of bytes copied.
+         * @throws IgniteCheckedException If any read operation failed.
+         */
+        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
+            assert dstOff >= 0;
+            assert dstOff + len <= dst.length : "Invalid indices [dst.length=" + dst.length + ", dstOff=" + dstOff +
+                ", len=" + len + ']';
+
+            int bytesCopied = 0;
+
+            if (first != null) {
+                bytesCopied += first.flatten(dst, pos, dstOff, len);
+
+                if (bytesCopied != len && second != null) {
+                    assert second.pos == first.pos + first.len;
+
+                    bytesCopied += second.flatten(dst, pos + bytesCopied, dstOff + bytesCopied, len - bytesCopied);
+                }
+            }
+
+            return bytesCopied;
+        }
+
+        /**
+         * Gets byte at specified position in buffer.
+         *
+         * @param pos Stream position.
+         * @return Read byte.
+         * @throws IgniteCheckedException If read failed.
+         */
+        public int atPosition(long pos) throws IgniteCheckedException {
+            // Should not reach here if stream contains no data.
+            assert first != null;
+
+            if (first.contains(pos)) {
+                byte[] bytes = first.readFut.get();
+
+                return bytes[((int)(pos - first.pos))] & 0xFF;
+            }
+            else {
+                assert second != null;
+                assert second.contains(pos);
+
+                byte[] bytes = second.readFut.get();
+
+                return bytes[((int)(pos - second.pos))] & 0xFF;
+            }
+        }
+
+        /**
+         * Starts asynchronous buffer refresh if needed, depending on current position.
+         *
+         * @param pos Current stream position.
+         */
+        public void refreshAhead(long pos) {
+            if (fullPrefetch(pos)) {
+                first = fetch(pos, bufHalfSize);
+                second = fetch(pos + bufHalfSize, bufHalfSize);
+            }
+            else if (needFlip(pos)) {
+                first = second;
+
+                second = fetch(first.pos + first.len, bufHalfSize);
+            }
+        }
+
+        /**
+         * @param pos Position from which read is expected.
+         * @return Number of bytes available to be read without blocking.
+         */
+        public int available(long pos) {
+            int available = 0;
+
+            if (first != null) {
+                if (first.contains(pos)) {
+                    if (first.ready()) {
+                        available += (pos - first.pos);
+
+                        if (second != null && second.ready())
+                            available += second.len;
+                    }
+                }
+                else {
+                    if (second != null && second.contains(pos) && second.ready())
+                        available += (pos - second.pos);
+                }
+            }
+
+            return available;
+        }
+
+        /**
+         * Checks if position shifted enough to forget previous buffer.
+         *
+         * @param pos Current position.
+         * @return {@code True} if need flip buffers.
+         */
+        private boolean needFlip(long pos) {
+            // Return true if we read more then half of second buffer.
+            return second != null && second.contains(pos);
+        }
+
+        /**
+         * Determines if all cached bytes should be discarded and new region should be
+         * prefetched.
+         *
+         * @param curPos Current stream position.
+         * @return {@code True} if need to refresh both blocks.
+         */
+        private boolean fullPrefetch(long curPos) {
+            // If no data was prefetched yet, return true.
+            return first == null || curPos < first.pos || (second != null && curPos >= second.pos + second.len);
+        }
+
+        /**
+         * Starts asynchronous fetch for given region.
+         *
+         * @param pos Position to read from.
+         * @param size Number of bytes to read.
+         * @return Fetch buffer part.
+         */
+        private FetchBufferPart fetch(long pos, int size) {
+            long remaining = limit - pos;
+
+            size = (int)Math.min(size, remaining);
+
+            return size <= 0 ? null :
+                new FetchBufferPart(delegate.hadoop().readData(delegate, pos, size, null, 0, 0), pos, size);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
new file mode 100644
index 0000000..4de4d53
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
@@ -0,0 +1,599 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.ipc.*;
+import org.apache.ignite.internal.util.ipc.shmem.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * IO layer implementation based on blocking IPC streams.
+ */
+@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+public class HadoopIpcIgfsIo implements HadoopIgfsIo {
+    /** Logger. */
+    private Log log;
+
+    /** Request futures map. */
+    private ConcurrentMap<Long, HadoopIgfsFuture> reqMap =
+        new ConcurrentHashMap8<>();
+
+    /** Request ID counter. */
+    private AtomicLong reqIdCnt = new AtomicLong();
+
+    /** Endpoint. */
+    private IpcEndpoint endpoint;
+
+    /** Endpoint output stream. */
+    private IgfsDataOutputStream out;
+
+    /** Protocol. */
+    private final IgfsMarshaller marsh;
+
+    /** Client reader thread. */
+    private Thread reader;
+
+    /** Lock for graceful shutdown. */
+    private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
+
+    /** Stopping flag. */
+    private volatile boolean stopping;
+
+    /** Server endpoint address. */
+    private final String endpointAddr;
+
+    /** Number of open file system sessions. */
+    private final AtomicInteger activeCnt = new AtomicInteger(1);
+
+    /** Event listeners. */
+    private final Collection<HadoopIgfsIpcIoListener> lsnrs =
+        new GridConcurrentHashSet<>();
+
+    /** Cached connections. */
+    private static final ConcurrentMap<String, HadoopIpcIgfsIo> ipcCache =
+        new ConcurrentHashMap8<>();
+
+    /** Striped lock that prevents multiple instance creation in {@link #get(Log, String)}. */
+    private static final GridStripedLock initLock = new GridStripedLock(32);
+
+    /**
+     * @param endpointAddr Endpoint.
+     * @param marsh Protocol.
+     * @param log Logger to use.
+     */
+    public HadoopIpcIgfsIo(String endpointAddr, IgfsMarshaller marsh, Log log) {
+        assert endpointAddr != null;
+        assert marsh != null;
+
+        this.endpointAddr = endpointAddr;
+        this.marsh = marsh;
+        this.log = log;
+    }
+
+    /**
+     * Returns a started and valid instance of this class
+     * for a given endpoint.
+     *
+     * @param log Logger to use for new instance.
+     * @param endpoint Endpoint string.
+     * @return New or existing cached instance, which is started and operational.
+     * @throws IOException If new instance was created but failed to start.
+     */
+    public static HadoopIpcIgfsIo get(Log log, String endpoint) throws IOException {
+        while (true) {
+            HadoopIpcIgfsIo clientIo = ipcCache.get(endpoint);
+
+            if (clientIo != null) {
+                if (clientIo.acquire())
+                    return clientIo;
+                else
+                    // If concurrent close.
+                    ipcCache.remove(endpoint, clientIo);
+            }
+            else {
+                Lock lock = initLock.getLock(endpoint);
+
+                lock.lock();
+
+                try {
+                    clientIo = ipcCache.get(endpoint);
+
+                    if (clientIo != null) { // Perform double check.
+                        if (clientIo.acquire())
+                            return clientIo;
+                        else
+                            // If concurrent close.
+                            ipcCache.remove(endpoint, clientIo);
+                    }
+
+                    // Otherwise try creating a new one.
+                    clientIo = new HadoopIpcIgfsIo(endpoint, new IgfsMarshaller(), log);
+
+                    try {
+                        clientIo.start();
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IOException(e.getMessage(), e);
+                    }
+
+                    HadoopIpcIgfsIo old = ipcCache.putIfAbsent(endpoint, clientIo);
+
+                    // Put in exclusive lock.
+                    assert old == null;
+
+                    return clientIo;
+                }
+                finally {
+                    lock.unlock();
+                }
+            }
+        }
+    }
+
+    /**
+     * Increases usage count for this instance.
+     *
+     * @return {@code true} if usage count is greater than zero.
+     */
+    private boolean acquire() {
+        while (true) {
+            int cnt = activeCnt.get();
+
+            if (cnt == 0) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO not acquired (count was 0): " + this);
+
+                return false;
+            }
+
+            // Need to make sure that no-one decremented count in between.
+            if (activeCnt.compareAndSet(cnt, cnt + 1)) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO acquired: " + this);
+
+                return true;
+            }
+        }
+    }
+
+    /**
+     * Releases this instance, decrementing usage count.
+     * <p>
+     * If usage count becomes zero, the instance is stopped
+     * and removed from cache.
+     */
+    public void release() {
+        while (true) {
+            int cnt = activeCnt.get();
+
+            if (cnt == 0) {
+                if (log.isDebugEnabled())
+                    log.debug("IPC IO not released (count was 0): " + this);
+
+                return;
+            }
+
+            if (activeCnt.compareAndSet(cnt, cnt - 1)) {
+                if (cnt == 1) {
+                    ipcCache.remove(endpointAddr, this);
+
+                    if (log.isDebugEnabled())
+                        log.debug("IPC IO stopping as unused: " + this);
+
+                    stop();
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("IPC IO released: " + this);
+
+                return;
+            }
+        }
+    }
+
+    /**
+     * Closes this IO instance, removing it from cache.
+     */
+    public void forceClose() {
+        if (ipcCache.remove(endpointAddr, this))
+            stop();
+    }
+
+    /**
+     * Starts the IO.
+     *
+     * @throws IgniteCheckedException If failed to connect the endpoint.
+     */
+    private void start() throws IgniteCheckedException {
+        boolean success = false;
+
+        try {
+            endpoint = IpcEndpointFactory.connectEndpoint(
+                    endpointAddr, new GridLoggerProxy(new HadoopIgfsJclLogger(log), null, null, ""));
+
+            out = new IgfsDataOutputStream(new BufferedOutputStream(endpoint.outputStream()));
+
+            reader = new ReaderThread();
+
+            // Required for Hadoop 2.x
+            reader.setDaemon(true);
+
+            reader.start();
+
+            success = true;
+        }
+        catch (IgniteCheckedException e) {
+            IpcOutOfSystemResourcesException resEx = e.getCause(IpcOutOfSystemResourcesException.class);
+
+            if (resEx != null)
+                throw new IgniteCheckedException(IpcSharedMemoryServerEndpoint.OUT_OF_RESOURCES_MSG, resEx);
+
+            throw e;
+        }
+        finally {
+            if (!success)
+                stop();
+        }
+    }
+
+    /**
+     * Shuts down the IO. No send requests will be accepted anymore, all pending futures will be failed.
+     * Close listeners will be invoked as if connection is closed by server.
+     */
+    private void stop() {
+        close0(null);
+
+        if (reader != null) {
+            try {
+                U.interrupt(reader);
+                U.join(reader);
+
+                reader = null;
+            }
+            catch (IgniteInterruptedCheckedException ignored) {
+                Thread.currentThread().interrupt();
+
+                log.warn("Got interrupted while waiting for reader thread to shut down (will return).");
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addEventListener(HadoopIgfsIpcIoListener lsnr) {
+        if (!busyLock.readLock().tryLock()) {
+            lsnr.onClose();
+
+            return;
+        }
+
+        boolean invokeNow = false;
+
+        try {
+            invokeNow = stopping;
+
+            if (!invokeNow)
+                lsnrs.add(lsnr);
+        }
+        finally {
+            busyLock.readLock().unlock();
+
+            if (invokeNow)
+                lsnr.onClose();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeEventListener(HadoopIgfsIpcIoListener lsnr) {
+        lsnrs.remove(lsnr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException {
+        return send(msg, null, 0, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff,
+        int outLen) throws IgniteCheckedException {
+        assert outBuf == null || msg.command() == IgfsIpcCommand.READ_BLOCK;
+
+        if (!busyLock.readLock().tryLock())
+            throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
+                "closed).");
+
+        try {
+            if (stopping)
+                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
+                    "closed).");
+
+            long reqId = reqIdCnt.getAndIncrement();
+
+            HadoopIgfsFuture<T> fut = new HadoopIgfsFuture<>();
+
+            fut.outputBuffer(outBuf);
+            fut.outputOffset(outOff);
+            fut.outputLength(outLen);
+            fut.read(msg.command() == IgfsIpcCommand.READ_BLOCK);
+
+            HadoopIgfsFuture oldFut = reqMap.putIfAbsent(reqId, fut);
+
+            assert oldFut == null;
+
+            if (log.isDebugEnabled())
+                log.debug("Sending IGFS message [reqId=" + reqId + ", msg=" + msg + ']');
+
+            byte[] hdr = IgfsMarshaller.createHeader(reqId, msg.command());
+
+            IgniteCheckedException err = null;
+
+            try {
+                synchronized (this) {
+                    marsh.marshall(msg, hdr, out);
+
+                    out.flush(); // Blocking operation + sometimes system call.
+                }
+            }
+            catch (IgniteCheckedException e) {
+                err = e;
+            }
+            catch (IOException e) {
+                err = new HadoopIgfsCommunicationException(e);
+            }
+
+            if (err != null) {
+                reqMap.remove(reqId, fut);
+
+                fut.onDone(err);
+            }
+
+            return fut;
+        }
+        finally {
+            busyLock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendPlain(IgfsMessage msg) throws IgniteCheckedException {
+        if (!busyLock.readLock().tryLock())
+            throw new HadoopIgfsCommunicationException("Failed to send message (client is being " +
+                "concurrently closed).");
+
+        try {
+            if (stopping)
+                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently closed).");
+
+            assert msg.command() == IgfsIpcCommand.WRITE_BLOCK;
+
+            IgfsStreamControlRequest req = (IgfsStreamControlRequest)msg;
+
+            byte[] hdr = IgfsMarshaller.createHeader(-1, IgfsIpcCommand.WRITE_BLOCK);
+
+            U.longToBytes(req.streamId(), hdr, 12);
+            U.intToBytes(req.length(), hdr, 20);
+
+            synchronized (this) {
+                out.write(hdr);
+                out.write(req.data(), (int)req.position(), req.length());
+
+                out.flush();
+            }
+        }
+        catch (IOException e) {
+            throw new HadoopIgfsCommunicationException(e);
+        }
+        finally {
+            busyLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Closes client but does not wait.
+     *
+     * @param err Error.
+     */
+    private void close0(@Nullable Throwable err) {
+        busyLock.writeLock().lock();
+
+        try {
+            if (stopping)
+                return;
+
+            stopping = true;
+        }
+        finally {
+            busyLock.writeLock().unlock();
+        }
+
+        if (err == null)
+            err = new IgniteCheckedException("Failed to perform request (connection was concurrently closed before response " +
+                "is received).");
+
+        // Clean up resources.
+        U.closeQuiet(out);
+
+        if (endpoint != null)
+            endpoint.close();
+
+        // Unwind futures. We can safely iterate here because no more futures will be added.
+        Iterator<HadoopIgfsFuture> it = reqMap.values().iterator();
+
+        while (it.hasNext()) {
+            HadoopIgfsFuture fut = it.next();
+
+            fut.onDone(err);
+
+            it.remove();
+        }
+
+        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
+            lsnr.onClose();
+    }
+
+    /**
+     * Do not extend {@code GridThread} to minimize class dependencies.
+     */
+    private class ReaderThread extends Thread {
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public void run() {
+            // Error to fail pending futures.
+            Throwable err = null;
+
+            try {
+                InputStream in = endpoint.inputStream();
+
+                IgfsDataInputStream dis = new IgfsDataInputStream(in);
+
+                byte[] hdr = new byte[IgfsMarshaller.HEADER_SIZE];
+                byte[] msgHdr = new byte[IgfsControlResponse.RES_HEADER_SIZE];
+
+                while (!Thread.currentThread().isInterrupted()) {
+                    dis.readFully(hdr);
+
+                    long reqId = U.bytesToLong(hdr, 0);
+
+                    // We don't wait for write responses, therefore reqId is -1.
+                    if (reqId == -1) {
+                        // We received a response which normally should not be sent. It must contain an error.
+                        dis.readFully(msgHdr);
+
+                        assert msgHdr[4] != 0;
+
+                        String errMsg = dis.readUTF();
+
+                        // Error code.
+                        dis.readInt();
+
+                        long streamId = dis.readLong();
+
+                        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
+                            lsnr.onError(streamId, errMsg);
+                    }
+                    else {
+                        HadoopIgfsFuture<Object> fut = reqMap.remove(reqId);
+
+                        if (fut == null) {
+                            String msg = "Failed to read response from server: response closure is unavailable for " +
+                                "requestId (will close connection):" + reqId;
+
+                            log.warn(msg);
+
+                            err = new IgniteCheckedException(msg);
+
+                            break;
+                        }
+                        else {
+                            try {
+                                IgfsIpcCommand cmd = IgfsIpcCommand.valueOf(U.bytesToInt(hdr, 8));
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Received IGFS response [reqId=" + reqId + ", cmd=" + cmd + ']');
+
+                                Object res = null;
+
+                                if (fut.read()) {
+                                    dis.readFully(msgHdr);
+
+                                    boolean hasErr = msgHdr[4] != 0;
+
+                                    if (hasErr) {
+                                        String errMsg = dis.readUTF();
+
+                                        // Error code.
+                                        Integer errCode = dis.readInt();
+
+                                        IgfsControlResponse.throwError(errCode, errMsg);
+                                    }
+
+                                    int blockLen = U.bytesToInt(msgHdr, 5);
+
+                                    int readLen = Math.min(blockLen, fut.outputLength());
+
+                                    if (readLen > 0) {
+                                        assert fut.outputBuffer() != null;
+
+                                        dis.readFully(fut.outputBuffer(), fut.outputOffset(), readLen);
+                                    }
+
+                                    if (readLen != blockLen) {
+                                        byte[] buf = new byte[blockLen - readLen];
+
+                                        dis.readFully(buf);
+
+                                        res = buf;
+                                    }
+                                }
+                                else
+                                    res = marsh.unmarshall(cmd, hdr, dis);
+
+                                fut.onDone(res);
+                            }
+                            catch (IgniteCheckedException e) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Failed to apply response closure (will fail request future): " +
+                                        e.getMessage());
+
+                                fut.onDone(e);
+
+                                err = e;
+                            }
+                        }
+                    }
+                }
+            }
+            catch (EOFException ignored) {
+                err = new IgniteCheckedException("Failed to read response from server (connection was closed by remote peer).");
+            }
+            catch (IOException e) {
+                if (!stopping)
+                    log.error("Failed to read data (connection will be closed)", e);
+
+                err = new HadoopIgfsCommunicationException(e);
+            }
+            catch (IgniteCheckedException e) {
+                if (!stopping)
+                    log.error("Failed to obtain endpoint input stream (connection will be closed)", e);
+
+                err = e;
+            }
+            finally {
+                close0(err);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return getClass().getSimpleName() + " [endpointAddr=" + endpointAddr + ", activeCnt=" + activeCnt +
+            ", stopping=" + stopping + ']';
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
new file mode 100644
index 0000000..19205c0
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
@@ -0,0 +1,466 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.*;
+
+/**
+ * Communication with external process (TCP or shmem).
+ */
+public class HadoopOutProcIgfs implements HadoopIgfsEx, HadoopIgfsIpcIoListener {
+    /** Expected result is boolean. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Boolean> BOOL_RES = createClosure();
+
+    /** Expected result is boolean. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Long> LONG_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsFile> FILE_RES = createClosure();
+
+    /** Expected result is {@code IgfsHandshakeResponse} */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        IgfsHandshakeResponse> HANDSHAKE_RES = createClosure();
+
+    /** Expected result is {@code IgfsStatus} */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsStatus> STATUS_RES =
+        createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        IgfsInputStreamDescriptor> STREAM_DESCRIPTOR_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsFile>> FILE_COL_RES = createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsPath>> PATH_COL_RES = createClosure();
+
+    /** Expected result is {@code IgfsPathSummary}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsPathSummary> SUMMARY_RES =
+        createClosure();
+
+    /** Expected result is {@code IgfsFile}. */
+    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
+        Collection<IgfsBlockLocation>> BLOCK_LOCATION_COL_RES = createClosure();
+
+    /** Grid name. */
+    private final String grid;
+
+    /** IGFS name. */
+    private final String igfs;
+
+    /** Client log. */
+    private final Log log;
+
+    /** Client IO. */
+    private final HadoopIpcIgfsIo io;
+
+    /** Event listeners. */
+    private final Map<Long, HadoopIgfsStreamEventListener> lsnrs = new ConcurrentHashMap8<>();
+
+    /**
+     * Constructor for TCP endpoint.
+     *
+     * @param host Host.
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    public HadoopOutProcIgfs(String host, int port, String grid, String igfs, Log log) throws IOException {
+        this(host, port, grid, igfs, false, log);
+    }
+
+    /**
+     * Constructor for shmem endpoint.
+     *
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    public HadoopOutProcIgfs(int port, String grid, String igfs, Log log) throws IOException {
+        this(null, port, grid, igfs, true, log);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param host Host.
+     * @param port Port.
+     * @param grid Grid name.
+     * @param igfs IGFS name.
+     * @param shmem Shared memory flag.
+     * @param log Client logger.
+     * @throws IOException If failed.
+     */
+    private HadoopOutProcIgfs(String host, int port, String grid, String igfs, boolean shmem, Log log)
+        throws IOException {
+        assert host != null && !shmem || host == null && shmem :
+            "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
+
+        String endpoint = host != null ? host + ":" + port : "shmem:" + port;
+
+        this.grid = grid;
+        this.igfs = igfs;
+        this.log = log;
+
+        io = HadoopIpcIgfsIo.get(log, endpoint);
+
+        io.addEventListener(this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException {
+        final IgfsHandshakeRequest req = new IgfsHandshakeRequest();
+
+        req.gridName(grid);
+        req.igfsName(igfs);
+        req.logDirectory(logDir);
+
+        return io.send(req).chain(HANDSHAKE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close(boolean force) {
+        assert io != null;
+
+        io.removeEventListener(this);
+
+        if (force)
+            io.forceClose();
+        else
+            io.release();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(INFO);
+        msg.path(path);
+
+        return io.send(msg).chain(FILE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(UPDATE);
+        msg.path(path);
+        msg.properties(props);
+
+        return io.send(msg).chain(FILE_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(SET_TIMES);
+        msg.path(path);
+        msg.accessTime(accessTime);
+        msg.modificationTime(modificationTime);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(RENAME);
+        msg.path(src);
+        msg.destinationPath(dest);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(DELETE);
+        msg.path(path);
+        msg.flag(recursive);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
+        throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(AFFINITY);
+        msg.path(path);
+        msg.start(start);
+        msg.length(len);
+
+        return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(PATH_SUMMARY);
+        msg.path(path);
+
+        return io.send(msg).chain(SUMMARY_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(MAKE_DIRECTORIES);
+        msg.path(path);
+        msg.properties(props);
+
+        return io.send(msg).chain(BOOL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(LIST_FILES);
+        msg.path(path);
+
+        return io.send(msg).chain(FILE_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(LIST_PATHS);
+        msg.path(path);
+
+        return io.send(msg).chain(PATH_COL_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
+        return io.send(new IgfsStatusRequest()).chain(STATUS_RES).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_READ);
+        msg.path(path);
+        msg.flag(false);
+
+        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate open(IgfsPath path,
+        int seqReadsBeforePrefetch) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_READ);
+        msg.path(path);
+        msg.flag(true);
+        msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
+
+        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
+        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_CREATE);
+        msg.path(path);
+        msg.flag(overwrite);
+        msg.colocate(colocate);
+        msg.properties(props);
+        msg.replication(replication);
+        msg.blockSize(blockSize);
+
+        Long streamId = io.send(msg).chain(LONG_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, streamId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
+        @Nullable Map<String, String> props) throws IgniteCheckedException {
+        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
+
+        msg.command(OPEN_APPEND);
+        msg.path(path);
+        msg.flag(create);
+        msg.properties(props);
+
+        Long streamId = io.send(msg).chain(LONG_RES).get();
+
+        return new HadoopIgfsStreamDelegate(this, streamId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridPlainFuture<byte[]> readData(HadoopIgfsStreamDelegate desc, long pos, int len,
+        final @Nullable byte[] outBuf, final int outOff, final int outLen) {
+        assert len > 0;
+
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(READ_BLOCK);
+        msg.streamId((long) desc.target());
+        msg.position(pos);
+        msg.length(len);
+
+        try {
+            return io.send(msg, outBuf, outOff, outLen);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridPlainFutureAdapter<>(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeData(HadoopIgfsStreamDelegate desc, byte[] data, int off, int len)
+        throws IOException {
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(WRITE_BLOCK);
+        msg.streamId((long) desc.target());
+        msg.data(data);
+        msg.position(off);
+        msg.length(len);
+
+        try {
+            io.sendPlain(msg);
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush(HadoopIgfsStreamDelegate delegate) throws IOException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void closeStream(HadoopIgfsStreamDelegate desc) throws IOException {
+        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
+
+        msg.command(CLOSE);
+        msg.streamId((long)desc.target());
+
+        try {
+            io.send(msg).chain(BOOL_RES).get();
+        }
+        catch (IgniteCheckedException e) {
+            throw HadoopIgfsUtils.cast(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addEventListener(HadoopIgfsStreamDelegate desc,
+        HadoopIgfsStreamEventListener lsnr) {
+        long streamId = desc.target();
+
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.put(streamId, lsnr);
+
+        assert lsnr0 == null || lsnr0 == lsnr;
+
+        if (log.isDebugEnabled())
+            log.debug("Added stream event listener [streamId=" + streamId + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeEventListener(HadoopIgfsStreamDelegate desc) {
+        long streamId = desc.target();
+
+        HadoopIgfsStreamEventListener lsnr0 = lsnrs.remove(streamId);
+
+        if (lsnr0 != null && log.isDebugEnabled())
+            log.debug("Removed stream event listener [streamId=" + streamId + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() {
+        for (HadoopIgfsStreamEventListener lsnr : lsnrs.values()) {
+            try {
+                lsnr.onClose();
+            }
+            catch (IgniteCheckedException e) {
+                log.warn("Got exception from stream event listener (will ignore): " + lsnr, e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(long streamId, String errMsg) {
+        HadoopIgfsStreamEventListener lsnr = lsnrs.get(streamId);
+
+        if (lsnr != null)
+            lsnr.onError(errMsg);
+        else
+            log.warn("Received write error response for not registered output stream (will ignore) " +
+                "[streamId= " + streamId + ']');
+    }
+
+    /**
+     * Creates conversion closure for given type.
+     *
+     * @param <T> Type of expected result.
+     * @return Conversion closure.
+     */
+    @SuppressWarnings("unchecked")
+    private static <T> GridPlainClosure<GridPlainFuture<IgfsMessage>, T> createClosure() {
+        return new GridPlainClosure<GridPlainFuture<IgfsMessage>, T>() {
+            @Override public T apply(GridPlainFuture<IgfsMessage> fut) throws IgniteCheckedException {
+                IgfsControlResponse res = (IgfsControlResponse)fut.get();
+
+                if (res.hasError())
+                    res.throwError();
+
+                return (T)res.response();
+            }
+        };
+    }
+}


[12/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
deleted file mode 100644
index 545c1b8..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
+++ /dev/null
@@ -1,593 +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.shuffle;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.collections.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.internal.util.worker.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.thread.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
-
-/**
- * Shuffle job.
- */
-public class GridHadoopShuffleJob<T> implements AutoCloseable {
-    /** */
-    private static final int MSG_BUF_SIZE = 128 * 1024;
-
-    /** */
-    private final GridHadoopJob job;
-
-    /** */
-    private final GridUnsafeMemory mem;
-
-    /** */
-    private final boolean needPartitioner;
-
-    /** Collection of task contexts for each reduce task. */
-    private final Map<Integer, GridHadoopTaskContext> reducersCtx = new HashMap<>();
-
-    /** Reducers addresses. */
-    private T[] reduceAddrs;
-
-    /** Local reducers address. */
-    private final T locReduceAddr;
-
-    /** */
-    private final GridHadoopShuffleMessage[] msgs;
-
-    /** */
-    private final AtomicReferenceArray<GridHadoopMultimap> maps;
-
-    /** */
-    private volatile IgniteInClosure2X<T, GridHadoopShuffleMessage> io;
-
-    /** */
-    protected ConcurrentMap<Long, IgniteBiTuple<GridHadoopShuffleMessage, GridFutureAdapterEx<?>>> sentMsgs =
-        new ConcurrentHashMap<>();
-
-    /** */
-    private volatile GridWorker snd;
-
-    /** Latch for remote addresses waiting. */
-    private final CountDownLatch ioInitLatch = new CountDownLatch(1);
-
-    /** Finished flag. Set on flush or close. */
-    private volatile boolean flushed;
-
-    /** */
-    private final IgniteLogger log;
-
-    /**
-     * @param locReduceAddr Local reducer address.
-     * @param log Logger.
-     * @param job Job.
-     * @param mem Memory.
-     * @param totalReducerCnt Amount of reducers in the Job.
-     * @param locReducers Reducers will work on current node.
-     * @throws IgniteCheckedException If error.
-     */
-    public GridHadoopShuffleJob(T locReduceAddr, IgniteLogger log, GridHadoopJob job, GridUnsafeMemory mem,
-        int totalReducerCnt, int[] locReducers) throws IgniteCheckedException {
-        this.locReduceAddr = locReduceAddr;
-        this.job = job;
-        this.mem = mem;
-        this.log = log.getLogger(GridHadoopShuffleJob.class);
-
-        if (!F.isEmpty(locReducers)) {
-            for (int rdc : locReducers) {
-                GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.REDUCE, job.id(), rdc, 0, null);
-
-                reducersCtx.put(rdc, job.getTaskContext(taskInfo));
-            }
-        }
-
-        needPartitioner = totalReducerCnt > 1;
-
-        maps = new AtomicReferenceArray<>(totalReducerCnt);
-        msgs = new GridHadoopShuffleMessage[totalReducerCnt];
-    }
-
-    /**
-     * @param reduceAddrs Addresses of reducers.
-     * @return {@code True} if addresses were initialized by this call.
-     */
-    public boolean initializeReduceAddresses(T[] reduceAddrs) {
-        if (this.reduceAddrs == null) {
-            this.reduceAddrs = reduceAddrs;
-
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * @return {@code True} if reducers addresses were initialized.
-     */
-    public boolean reducersInitialized() {
-        return reduceAddrs != null;
-    }
-
-    /**
-     * @param gridName Grid name.
-     * @param io IO Closure for sending messages.
-     */
-    @SuppressWarnings("BusyWait")
-    public void startSending(String gridName, IgniteInClosure2X<T, GridHadoopShuffleMessage> io) {
-        assert snd == null;
-        assert io != null;
-
-        this.io = io;
-
-        if (!flushed) {
-            snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
-                @Override protected void body() throws InterruptedException {
-                    try {
-                        while (!isCancelled()) {
-                            Thread.sleep(5);
-
-                            collectUpdatesAndSend(false);
-                        }
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            };
-
-            new IgniteThread(snd).start();
-        }
-
-        ioInitLatch.countDown();
-    }
-
-    /**
-     * @param maps Maps.
-     * @param idx Index.
-     * @return Map.
-     */
-    private GridHadoopMultimap getOrCreateMap(AtomicReferenceArray<GridHadoopMultimap> maps, int idx) {
-        GridHadoopMultimap map = maps.get(idx);
-
-        if (map == null) { // Create new map.
-            map = get(job.info(), SHUFFLE_REDUCER_NO_SORTING, false) ?
-                new GridHadoopConcurrentHashMultimap(job.info(), mem, get(job.info(), PARTITION_HASHMAP_SIZE, 8 * 1024)):
-                new GridHadoopSkipList(job.info(), mem);
-
-            if (!maps.compareAndSet(idx, null, map)) {
-                map.close();
-
-                return maps.get(idx);
-            }
-        }
-
-        return map;
-    }
-
-    /**
-     * @param msg Message.
-     * @throws IgniteCheckedException Exception.
-     */
-    public void onShuffleMessage(GridHadoopShuffleMessage msg) throws IgniteCheckedException {
-        assert msg.buffer() != null;
-        assert msg.offset() > 0;
-
-        GridHadoopTaskContext taskCtx = reducersCtx.get(msg.reducer());
-
-        GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(taskCtx.counters(), null);
-
-        perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis());
-
-        GridHadoopMultimap map = getOrCreateMap(maps, msg.reducer());
-
-        // Add data from message to the map.
-        try (GridHadoopMultimap.Adder adder = map.startAdding(taskCtx)) {
-            final GridUnsafeDataInput dataInput = new GridUnsafeDataInput();
-            final UnsafeValue val = new UnsafeValue(msg.buffer());
-
-            msg.visit(new GridHadoopShuffleMessage.Visitor() {
-                /** */
-                private GridHadoopMultimap.Key key;
-
-                @Override public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException {
-                    dataInput.bytes(buf, off, off + len);
-
-                    key = adder.addKey(dataInput, key);
-                }
-
-                @Override public void onValue(byte[] buf, int off, int len) {
-                    val.off = off;
-                    val.size = len;
-
-                    key.add(val);
-                }
-            });
-        }
-    }
-
-    /**
-     * @param ack Shuffle ack.
-     */
-    @SuppressWarnings("ConstantConditions")
-    public void onShuffleAck(GridHadoopShuffleAck ack) {
-        IgniteBiTuple<GridHadoopShuffleMessage, GridFutureAdapterEx<?>> tup = sentMsgs.get(ack.id());
-
-        if (tup != null)
-            tup.get2().onDone();
-        else
-            log.warning("Received shuffle ack for not registered shuffle id: " + ack);
-    }
-
-    /**
-     * Unsafe value.
-     */
-    private static class UnsafeValue implements GridHadoopMultimap.Value {
-        /** */
-        private final byte[] buf;
-
-        /** */
-        private int off;
-
-        /** */
-        private int size;
-
-        /**
-         * @param buf Buffer.
-         */
-        private UnsafeValue(byte[] buf) {
-            assert buf != null;
-
-            this.buf = buf;
-        }
-
-        /** */
-        @Override public int size() {
-            return size;
-        }
-
-        /** */
-        @Override public void copyTo(long ptr) {
-            UNSAFE.copyMemory(buf, BYTE_ARR_OFF + off, null, ptr, size);
-        }
-    }
-
-    /**
-     * Sends map updates to remote reducers.
-     */
-    private void collectUpdatesAndSend(boolean flush) throws IgniteCheckedException {
-        for (int i = 0; i < maps.length(); i++) {
-            GridHadoopMultimap map = maps.get(i);
-
-            if (map == null || locReduceAddr.equals(reduceAddrs[i]))
-                continue; // Skip empty map and local node.
-
-            if (msgs[i] == null)
-                msgs[i] = new GridHadoopShuffleMessage(job.id(), i, MSG_BUF_SIZE);
-
-            final int idx = i;
-
-            map.visit(false, new GridHadoopMultimap.Visitor() {
-                /** */
-                private long keyPtr;
-
-                /** */
-                private int keySize;
-
-                /** */
-                private boolean keyAdded;
-
-                /** {@inheritDoc} */
-                @Override public void onKey(long keyPtr, int keySize) {
-                    this.keyPtr = keyPtr;
-                    this.keySize = keySize;
-
-                    keyAdded = false;
-                }
-
-                private boolean tryAdd(long valPtr, int valSize) {
-                    GridHadoopShuffleMessage msg = msgs[idx];
-
-                    if (!keyAdded) { // Add key and value.
-                        int size = keySize + valSize;
-
-                        if (!msg.available(size, false))
-                            return false;
-
-                        msg.addKey(keyPtr, keySize);
-                        msg.addValue(valPtr, valSize);
-
-                        keyAdded = true;
-
-                        return true;
-                    }
-
-                    if (!msg.available(valSize, true))
-                        return false;
-
-                    msg.addValue(valPtr, valSize);
-
-                    return true;
-                }
-
-                /** {@inheritDoc} */
-                @Override public void onValue(long valPtr, int valSize) {
-                    if (tryAdd(valPtr, valSize))
-                        return;
-
-                    send(idx, keySize + valSize);
-
-                    keyAdded = false;
-
-                    if (!tryAdd(valPtr, valSize))
-                        throw new IllegalStateException();
-                }
-            });
-
-            if (flush && msgs[i].offset() != 0)
-                send(i, 0);
-        }
-    }
-
-    /**
-     * @param idx Index of message.
-     * @param newBufMinSize Min new buffer size.
-     */
-    private void send(final int idx, int newBufMinSize) {
-        final GridFutureAdapterEx<?> fut = new GridFutureAdapterEx<>();
-
-        GridHadoopShuffleMessage msg = msgs[idx];
-
-        final long msgId = msg.id();
-
-        IgniteBiTuple<GridHadoopShuffleMessage, GridFutureAdapterEx<?>> old = sentMsgs.putIfAbsent(msgId,
-            new IgniteBiTuple<GridHadoopShuffleMessage, GridFutureAdapterEx<?>>(msg, fut));
-
-        assert old == null;
-
-        try {
-            io.apply(reduceAddrs[idx], msg);
-        }
-        catch (GridClosureException e) {
-            fut.onDone(U.unwrap(e));
-        }
-
-        fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
-            @Override public void apply(IgniteInternalFuture<?> f) {
-                try {
-                    f.get();
-
-                    // Clean up the future from map only if there was no exception.
-                    // Otherwise flush() should fail.
-                    sentMsgs.remove(msgId);
-                }
-                catch (IgniteCheckedException e) {
-                    log.error("Failed to send message.", e);
-                }
-            }
-        });
-
-        msgs[idx] = newBufMinSize == 0 ? null : new GridHadoopShuffleMessage(job.id(), idx,
-            Math.max(MSG_BUF_SIZE, newBufMinSize));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
-        if (snd != null) {
-            snd.cancel();
-
-            try {
-                snd.join();
-            }
-            catch (InterruptedException e) {
-                throw new IgniteInterruptedCheckedException(e);
-            }
-        }
-
-        close(maps);
-    }
-
-    /**
-     * @param maps Maps.
-     */
-    private void close(AtomicReferenceArray<GridHadoopMultimap> maps) {
-        for (int i = 0; i < maps.length(); i++) {
-            GridHadoopMultimap map = maps.get(i);
-
-            if (map != null)
-                map.close();
-        }
-    }
-
-    /**
-     * @return Future.
-     */
-    @SuppressWarnings("unchecked")
-    public IgniteInternalFuture<?> flush() throws IgniteCheckedException {
-        if (log.isDebugEnabled())
-            log.debug("Flushing job " + job.id() + " on address " + locReduceAddr);
-
-        flushed = true;
-
-        if (maps.length() == 0)
-            return new GridFinishedFutureEx<>();
-
-        U.await(ioInitLatch);
-
-        GridWorker snd0 = snd;
-
-        if (snd0 != null) {
-            if (log.isDebugEnabled())
-                log.debug("Cancelling sender thread.");
-
-            snd0.cancel();
-
-            try {
-                snd0.join();
-
-                if (log.isDebugEnabled())
-                    log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
-            }
-            catch (InterruptedException e) {
-                throw new IgniteInterruptedCheckedException(e);
-            }
-        }
-
-        collectUpdatesAndSend(true); // With flush.
-
-        if (log.isDebugEnabled())
-            log.debug("Finished sending collected updates to remote reducers: " + job.id());
-
-        GridCompoundFuture fut = new GridCompoundFuture<>();
-
-        for (IgniteBiTuple<GridHadoopShuffleMessage, GridFutureAdapterEx<?>> tup : sentMsgs.values())
-            fut.add(tup.get2());
-
-        fut.markInitialized();
-
-        if (log.isDebugEnabled())
-            log.debug("Collected futures to compound futures for flush: " + sentMsgs.size());
-
-        return fut;
-    }
-
-    /**
-     * @param taskCtx Task context.
-     * @return Output.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridHadoopTaskOutput output(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        switch (taskCtx.taskInfo().type()) {
-            case MAP:
-                assert !job.info().hasCombiner() : "The output creation is allowed if combiner has not been defined.";
-
-            case COMBINE:
-                return new PartitionedOutput(taskCtx);
-
-            default:
-                throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type());
-        }
-    }
-
-    /**
-     * @param taskCtx Task context.
-     * @return Input.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        switch (taskCtx.taskInfo().type()) {
-            case REDUCE:
-                int reducer = taskCtx.taskInfo().taskNumber();
-
-                GridHadoopMultimap m = maps.get(reducer);
-
-                if (m != null)
-                    return m.input(taskCtx);
-
-                return new GridHadoopTaskInput() { // Empty input.
-                    @Override public boolean next() {
-                        return false;
-                    }
-
-                    @Override public Object key() {
-                        throw new IllegalStateException();
-                    }
-
-                    @Override public Iterator<?> values() {
-                        throw new IllegalStateException();
-                    }
-
-                    @Override public void close() {
-                        // No-op.
-                    }
-                };
-
-            default:
-                throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type());
-        }
-    }
-
-    /**
-     * Partitioned output.
-     */
-    private class PartitionedOutput implements GridHadoopTaskOutput {
-        /** */
-        private final GridHadoopTaskOutput[] adders = new GridHadoopTaskOutput[maps.length()];
-
-        /** */
-        private GridHadoopPartitioner partitioner;
-
-        /** */
-        private final GridHadoopTaskContext taskCtx;
-
-        /**
-         * Constructor.
-         * @param taskCtx Task context.
-         */
-        private PartitionedOutput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-            this.taskCtx = taskCtx;
-
-            if (needPartitioner)
-                partitioner = taskCtx.partitioner();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Object key, Object val) throws IgniteCheckedException {
-            int part = 0;
-
-            if (partitioner != null) {
-                part = partitioner.partition(key, val, adders.length);
-
-                if (part < 0 || part >= adders.length)
-                    throw new IgniteCheckedException("Invalid partition: " + part);
-            }
-
-            GridHadoopTaskOutput out = adders[part];
-
-            if (out == null)
-                adders[part] = out = getOrCreateMap(maps, part).startAdding(taskCtx);
-
-            out.write(key, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            for (GridHadoopTaskOutput adder : adders) {
-                if (adder != null)
-                    adder.close();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleMessage.java
deleted file mode 100644
index 24ebc0c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleMessage.java
+++ /dev/null
@@ -1,242 +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.shuffle;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
-
-/**
- * Shuffle message.
- */
-public class GridHadoopShuffleMessage implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final AtomicLong ids = new AtomicLong();
-
-    /** */
-    private static final byte MARKER_KEY = (byte)17;
-
-    /** */
-    private static final byte MARKER_VALUE = (byte)31;
-
-    /** */
-    @GridToStringInclude
-    private long msgId;
-
-    /** */
-    @GridToStringInclude
-    private GridHadoopJobId jobId;
-
-    /** */
-    @GridToStringInclude
-    private int reducer;
-
-    /** */
-    private byte[] buf;
-
-    /** */
-    @GridToStringInclude
-    private int off;
-
-    /**
-     *
-     */
-    public GridHadoopShuffleMessage() {
-        // No-op.
-    }
-
-    /**
-     * @param size Size.
-     */
-    public GridHadoopShuffleMessage(GridHadoopJobId jobId, int reducer, int size) {
-        assert jobId != null;
-
-        buf = new byte[size];
-
-        this.jobId = jobId;
-        this.reducer = reducer;
-
-        msgId = ids.incrementAndGet();
-    }
-
-    /**
-     * @return Message ID.
-     */
-    public long id() {
-        return msgId;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @return Reducer.
-     */
-    public int reducer() {
-        return reducer;
-    }
-
-    /**
-     * @return Buffer.
-     */
-    public byte[] buffer() {
-        return buf;
-    }
-
-    /**
-     * @return Offset.
-     */
-    public int offset() {
-        return off;
-    }
-
-    /**
-     * @param size Size.
-     * @param valOnly Only value wll be added.
-     * @return {@code true} If this message can fit additional data of this size
-     */
-    public boolean available(int size, boolean valOnly) {
-        size += valOnly ? 5 : 10;
-
-        if (off + size > buf.length) {
-            if (off == 0) { // Resize if requested size is too big.
-                buf = new byte[size];
-
-                return true;
-            }
-
-            return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * @param keyPtr Key pointer.
-     * @param keySize Key size.
-     */
-    public void addKey(long keyPtr, int keySize) {
-        add(MARKER_KEY, keyPtr, keySize);
-    }
-
-    /**
-     * @param valPtr Value pointer.
-     * @param valSize Value size.
-     */
-    public void addValue(long valPtr, int valSize) {
-        add(MARKER_VALUE, valPtr, valSize);
-    }
-
-    /**
-     * @param marker Marker.
-     * @param ptr Pointer.
-     * @param size Size.
-     */
-    private void add(byte marker, long ptr, int size) {
-        buf[off++] = marker;
-
-        UNSAFE.putInt(buf, BYTE_ARR_OFF + off, size);
-
-        off += 4;
-
-        UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF + off, size);
-
-        off += size;
-    }
-
-    /**
-     * @param v Visitor.
-     */
-    public void visit(Visitor v) throws IgniteCheckedException {
-        for (int i = 0; i < off;) {
-            byte marker = buf[i++];
-
-            int size = UNSAFE.getInt(buf, BYTE_ARR_OFF + i);
-
-            i += 4;
-
-            if (marker == MARKER_VALUE)
-                v.onValue(buf, i, size);
-            else if (marker == MARKER_KEY)
-                v.onKey(buf, i, size);
-            else
-                throw new IllegalStateException();
-
-            i += size;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        jobId.writeExternal(out);
-        out.writeLong(msgId);
-        out.writeInt(reducer);
-        out.writeInt(off);
-        U.writeByteArray(out, buf);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = new GridHadoopJobId();
-
-        jobId.readExternal(in);
-        msgId = in.readLong();
-        reducer = in.readInt();
-        off = in.readInt();
-        buf = U.readByteArray(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopShuffleMessage.class, this);
-    }
-
-    /**
-     * Visitor.
-     */
-    public static interface Visitor {
-        /**
-         * @param buf Buffer.
-         * @param off Offset.
-         * @param len Length.
-         */
-        public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException;
-
-        /**
-         * @param buf Buffer.
-         * @param off Offset.
-         * @param len Length.
-         */
-        public void onValue(byte[] buf, int off, int len) throws IgniteCheckedException;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 9880093..267316e 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -37,7 +37,7 @@ import java.util.concurrent.*;
  */
 public class HadoopShuffle extends HadoopComponent {
     /** */
-    private final ConcurrentMap<GridHadoopJobId, GridHadoopShuffleJob<UUID>> jobs = new ConcurrentHashMap<>();
+    private final ConcurrentMap<GridHadoopJobId, HadoopShuffleJob<UUID>> jobs = new ConcurrentHashMap<>();
 
     /** */
     protected final GridUnsafeMemory mem = new GridUnsafeMemory(0);
@@ -49,7 +49,7 @@ public class HadoopShuffle extends HadoopComponent {
         ctx.kernalContext().io().addUserMessageListener(GridTopic.TOPIC_HADOOP,
             new IgniteBiPredicate<UUID, Object>() {
                 @Override public boolean apply(UUID nodeId, Object msg) {
-                    return onMessageReceived(nodeId, (GridHadoopMessage)msg);
+                    return onMessageReceived(nodeId, (HadoopMessage)msg);
                 }
             });
     }
@@ -60,7 +60,7 @@ public class HadoopShuffle extends HadoopComponent {
      * @param cancel If should cancel all ongoing activities.
      */
     @Override public void stop(boolean cancel) {
-        for (GridHadoopShuffleJob job : jobs.values()) {
+        for (HadoopShuffleJob job : jobs.values()) {
             try {
                 job.close();
             }
@@ -79,10 +79,10 @@ public class HadoopShuffle extends HadoopComponent {
      * @return Created shuffle job.
      * @throws IgniteCheckedException If job creation failed.
      */
-    private GridHadoopShuffleJob<UUID> newJob(GridHadoopJobId jobId) throws IgniteCheckedException {
+    private HadoopShuffleJob<UUID> newJob(GridHadoopJobId jobId) throws IgniteCheckedException {
         GridHadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
 
-        GridHadoopShuffleJob<UUID> job = new GridHadoopShuffleJob<>(ctx.localNodeId(), log,
+        HadoopShuffleJob<UUID> job = new HadoopShuffleJob<>(ctx.localNodeId(), log,
             ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId()));
 
         UUID[] rdcAddrs = new UUID[plan.reducers()];
@@ -117,13 +117,13 @@ public class HadoopShuffle extends HadoopComponent {
      * @param jobId Task info.
      * @return Shuffle job.
      */
-    private GridHadoopShuffleJob<UUID> job(GridHadoopJobId jobId) throws IgniteCheckedException {
-        GridHadoopShuffleJob<UUID> res = jobs.get(jobId);
+    private HadoopShuffleJob<UUID> job(GridHadoopJobId jobId) throws IgniteCheckedException {
+        HadoopShuffleJob<UUID> res = jobs.get(jobId);
 
         if (res == null) {
             res = newJob(jobId);
 
-            GridHadoopShuffleJob<UUID> old = jobs.putIfAbsent(jobId, res);
+            HadoopShuffleJob<UUID> old = jobs.putIfAbsent(jobId, res);
 
             if (old != null) {
                 res.close();
@@ -142,10 +142,10 @@ public class HadoopShuffle extends HadoopComponent {
      *
      * @param shuffleJob Job to start sending for.
      */
-    private void startSending(GridHadoopShuffleJob<UUID> shuffleJob) {
+    private void startSending(HadoopShuffleJob<UUID> shuffleJob) {
         shuffleJob.startSending(ctx.kernalContext().gridName(),
-            new IgniteInClosure2X<UUID, GridHadoopShuffleMessage>() {
-                @Override public void applyx(UUID dest, GridHadoopShuffleMessage msg) throws IgniteCheckedException {
+            new IgniteInClosure2X<UUID, HadoopShuffleMessage>() {
+                @Override public void applyx(UUID dest, HadoopShuffleMessage msg) throws IgniteCheckedException {
                     send0(dest, msg);
                 }
             }
@@ -159,9 +159,9 @@ public class HadoopShuffle extends HadoopComponent {
      * @param msg Received message.
      * @return {@code True}.
      */
-    public boolean onMessageReceived(UUID src, GridHadoopMessage msg) {
-        if (msg instanceof GridHadoopShuffleMessage) {
-            GridHadoopShuffleMessage m = (GridHadoopShuffleMessage)msg;
+    public boolean onMessageReceived(UUID src, HadoopMessage msg) {
+        if (msg instanceof HadoopShuffleMessage) {
+            HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
 
             try {
                 job(m.jobId()).onShuffleMessage(m);
@@ -172,14 +172,14 @@ public class HadoopShuffle extends HadoopComponent {
 
             try {
                 // Reply with ack.
-                send0(src, new GridHadoopShuffleAck(m.id(), m.jobId()));
+                send0(src, new HadoopShuffleAck(m.id(), m.jobId()));
             }
             catch (IgniteCheckedException e) {
                 U.error(log, "Failed to reply back to shuffle message sender [snd=" + src + ", msg=" + msg + ']', e);
             }
         }
-        else if (msg instanceof GridHadoopShuffleAck) {
-            GridHadoopShuffleAck m = (GridHadoopShuffleAck)msg;
+        else if (msg instanceof HadoopShuffleAck) {
+            HadoopShuffleAck m = (HadoopShuffleAck)msg;
 
             try {
                 job(m.jobId()).onShuffleAck(m);
@@ -215,7 +215,7 @@ public class HadoopShuffle extends HadoopComponent {
      * @param jobId Job id.
      */
     public void jobFinished(GridHadoopJobId jobId) {
-        GridHadoopShuffleJob job = jobs.remove(jobId);
+        HadoopShuffleJob job = jobs.remove(jobId);
 
         if (job != null) {
             try {
@@ -234,7 +234,7 @@ public class HadoopShuffle extends HadoopComponent {
      * @return Future.
      */
     public IgniteInternalFuture<?> flush(GridHadoopJobId jobId) {
-        GridHadoopShuffleJob job = jobs.get(jobId);
+        HadoopShuffleJob job = jobs.get(jobId);
 
         if (job == null)
             return new GridFinishedFutureEx<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java
new file mode 100644
index 0000000..53ff2d1
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleAck.java
@@ -0,0 +1,91 @@
+/*
+ * 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.shuffle;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Acknowledgement message.
+ */
+public class HadoopShuffleAck implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    @GridToStringInclude
+    private long msgId;
+
+    /** */
+    @GridToStringInclude
+    private GridHadoopJobId jobId;
+
+    /**
+     *
+     */
+    public HadoopShuffleAck() {
+        // No-op.
+    }
+
+    /**
+     * @param msgId Message ID.
+     */
+    public HadoopShuffleAck(long msgId, GridHadoopJobId jobId) {
+        assert jobId != null;
+
+        this.msgId = msgId;
+        this.jobId = jobId;
+    }
+
+    /**
+     * @return Message ID.
+     */
+    public long id() {
+        return msgId;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+        out.writeLong(msgId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new GridHadoopJobId();
+
+        jobId.readExternal(in);
+        msgId = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopShuffleAck.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
new file mode 100644
index 0000000..a75b34b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -0,0 +1,593 @@
+/*
+ * 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.shuffle;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.collections.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.thread.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
+import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
+
+/**
+ * Shuffle job.
+ */
+public class HadoopShuffleJob<T> implements AutoCloseable {
+    /** */
+    private static final int MSG_BUF_SIZE = 128 * 1024;
+
+    /** */
+    private final GridHadoopJob job;
+
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /** */
+    private final boolean needPartitioner;
+
+    /** Collection of task contexts for each reduce task. */
+    private final Map<Integer, GridHadoopTaskContext> reducersCtx = new HashMap<>();
+
+    /** Reducers addresses. */
+    private T[] reduceAddrs;
+
+    /** Local reducers address. */
+    private final T locReduceAddr;
+
+    /** */
+    private final HadoopShuffleMessage[] msgs;
+
+    /** */
+    private final AtomicReferenceArray<HadoopMultimap> maps;
+
+    /** */
+    private volatile IgniteInClosure2X<T, HadoopShuffleMessage> io;
+
+    /** */
+    protected ConcurrentMap<Long, IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>>> sentMsgs =
+        new ConcurrentHashMap<>();
+
+    /** */
+    private volatile GridWorker snd;
+
+    /** Latch for remote addresses waiting. */
+    private final CountDownLatch ioInitLatch = new CountDownLatch(1);
+
+    /** Finished flag. Set on flush or close. */
+    private volatile boolean flushed;
+
+    /** */
+    private final IgniteLogger log;
+
+    /**
+     * @param locReduceAddr Local reducer address.
+     * @param log Logger.
+     * @param job Job.
+     * @param mem Memory.
+     * @param totalReducerCnt Amount of reducers in the Job.
+     * @param locReducers Reducers will work on current node.
+     * @throws IgniteCheckedException If error.
+     */
+    public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, GridHadoopJob job, GridUnsafeMemory mem,
+        int totalReducerCnt, int[] locReducers) throws IgniteCheckedException {
+        this.locReduceAddr = locReduceAddr;
+        this.job = job;
+        this.mem = mem;
+        this.log = log.getLogger(HadoopShuffleJob.class);
+
+        if (!F.isEmpty(locReducers)) {
+            for (int rdc : locReducers) {
+                GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(GridHadoopTaskType.REDUCE, job.id(), rdc, 0, null);
+
+                reducersCtx.put(rdc, job.getTaskContext(taskInfo));
+            }
+        }
+
+        needPartitioner = totalReducerCnt > 1;
+
+        maps = new AtomicReferenceArray<>(totalReducerCnt);
+        msgs = new HadoopShuffleMessage[totalReducerCnt];
+    }
+
+    /**
+     * @param reduceAddrs Addresses of reducers.
+     * @return {@code True} if addresses were initialized by this call.
+     */
+    public boolean initializeReduceAddresses(T[] reduceAddrs) {
+        if (this.reduceAddrs == null) {
+            this.reduceAddrs = reduceAddrs;
+
+            return true;
+        }
+
+        return false;
+    }
+
+    /**
+     * @return {@code True} if reducers addresses were initialized.
+     */
+    public boolean reducersInitialized() {
+        return reduceAddrs != null;
+    }
+
+    /**
+     * @param gridName Grid name.
+     * @param io IO Closure for sending messages.
+     */
+    @SuppressWarnings("BusyWait")
+    public void startSending(String gridName, IgniteInClosure2X<T, HadoopShuffleMessage> io) {
+        assert snd == null;
+        assert io != null;
+
+        this.io = io;
+
+        if (!flushed) {
+            snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
+                @Override protected void body() throws InterruptedException {
+                    try {
+                        while (!isCancelled()) {
+                            Thread.sleep(5);
+
+                            collectUpdatesAndSend(false);
+                        }
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw new IllegalStateException(e);
+                    }
+                }
+            };
+
+            new IgniteThread(snd).start();
+        }
+
+        ioInitLatch.countDown();
+    }
+
+    /**
+     * @param maps Maps.
+     * @param idx Index.
+     * @return Map.
+     */
+    private HadoopMultimap getOrCreateMap(AtomicReferenceArray<HadoopMultimap> maps, int idx) {
+        HadoopMultimap map = maps.get(idx);
+
+        if (map == null) { // Create new map.
+            map = get(job.info(), SHUFFLE_REDUCER_NO_SORTING, false) ?
+                new HadoopConcurrentHashMultimap(job.info(), mem, get(job.info(), PARTITION_HASHMAP_SIZE, 8 * 1024)):
+                new HadoopSkipList(job.info(), mem);
+
+            if (!maps.compareAndSet(idx, null, map)) {
+                map.close();
+
+                return maps.get(idx);
+            }
+        }
+
+        return map;
+    }
+
+    /**
+     * @param msg Message.
+     * @throws IgniteCheckedException Exception.
+     */
+    public void onShuffleMessage(HadoopShuffleMessage msg) throws IgniteCheckedException {
+        assert msg.buffer() != null;
+        assert msg.offset() > 0;
+
+        GridHadoopTaskContext taskCtx = reducersCtx.get(msg.reducer());
+
+        HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(taskCtx.counters(), null);
+
+        perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis());
+
+        HadoopMultimap map = getOrCreateMap(maps, msg.reducer());
+
+        // Add data from message to the map.
+        try (HadoopMultimap.Adder adder = map.startAdding(taskCtx)) {
+            final GridUnsafeDataInput dataInput = new GridUnsafeDataInput();
+            final UnsafeValue val = new UnsafeValue(msg.buffer());
+
+            msg.visit(new HadoopShuffleMessage.Visitor() {
+                /** */
+                private HadoopMultimap.Key key;
+
+                @Override public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException {
+                    dataInput.bytes(buf, off, off + len);
+
+                    key = adder.addKey(dataInput, key);
+                }
+
+                @Override public void onValue(byte[] buf, int off, int len) {
+                    val.off = off;
+                    val.size = len;
+
+                    key.add(val);
+                }
+            });
+        }
+    }
+
+    /**
+     * @param ack Shuffle ack.
+     */
+    @SuppressWarnings("ConstantConditions")
+    public void onShuffleAck(HadoopShuffleAck ack) {
+        IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>> tup = sentMsgs.get(ack.id());
+
+        if (tup != null)
+            tup.get2().onDone();
+        else
+            log.warning("Received shuffle ack for not registered shuffle id: " + ack);
+    }
+
+    /**
+     * Unsafe value.
+     */
+    private static class UnsafeValue implements HadoopMultimap.Value {
+        /** */
+        private final byte[] buf;
+
+        /** */
+        private int off;
+
+        /** */
+        private int size;
+
+        /**
+         * @param buf Buffer.
+         */
+        private UnsafeValue(byte[] buf) {
+            assert buf != null;
+
+            this.buf = buf;
+        }
+
+        /** */
+        @Override public int size() {
+            return size;
+        }
+
+        /** */
+        @Override public void copyTo(long ptr) {
+            UNSAFE.copyMemory(buf, BYTE_ARR_OFF + off, null, ptr, size);
+        }
+    }
+
+    /**
+     * Sends map updates to remote reducers.
+     */
+    private void collectUpdatesAndSend(boolean flush) throws IgniteCheckedException {
+        for (int i = 0; i < maps.length(); i++) {
+            HadoopMultimap map = maps.get(i);
+
+            if (map == null || locReduceAddr.equals(reduceAddrs[i]))
+                continue; // Skip empty map and local node.
+
+            if (msgs[i] == null)
+                msgs[i] = new HadoopShuffleMessage(job.id(), i, MSG_BUF_SIZE);
+
+            final int idx = i;
+
+            map.visit(false, new HadoopMultimap.Visitor() {
+                /** */
+                private long keyPtr;
+
+                /** */
+                private int keySize;
+
+                /** */
+                private boolean keyAdded;
+
+                /** {@inheritDoc} */
+                @Override public void onKey(long keyPtr, int keySize) {
+                    this.keyPtr = keyPtr;
+                    this.keySize = keySize;
+
+                    keyAdded = false;
+                }
+
+                private boolean tryAdd(long valPtr, int valSize) {
+                    HadoopShuffleMessage msg = msgs[idx];
+
+                    if (!keyAdded) { // Add key and value.
+                        int size = keySize + valSize;
+
+                        if (!msg.available(size, false))
+                            return false;
+
+                        msg.addKey(keyPtr, keySize);
+                        msg.addValue(valPtr, valSize);
+
+                        keyAdded = true;
+
+                        return true;
+                    }
+
+                    if (!msg.available(valSize, true))
+                        return false;
+
+                    msg.addValue(valPtr, valSize);
+
+                    return true;
+                }
+
+                /** {@inheritDoc} */
+                @Override public void onValue(long valPtr, int valSize) {
+                    if (tryAdd(valPtr, valSize))
+                        return;
+
+                    send(idx, keySize + valSize);
+
+                    keyAdded = false;
+
+                    if (!tryAdd(valPtr, valSize))
+                        throw new IllegalStateException();
+                }
+            });
+
+            if (flush && msgs[i].offset() != 0)
+                send(i, 0);
+        }
+    }
+
+    /**
+     * @param idx Index of message.
+     * @param newBufMinSize Min new buffer size.
+     */
+    private void send(final int idx, int newBufMinSize) {
+        final GridFutureAdapterEx<?> fut = new GridFutureAdapterEx<>();
+
+        HadoopShuffleMessage msg = msgs[idx];
+
+        final long msgId = msg.id();
+
+        IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>> old = sentMsgs.putIfAbsent(msgId,
+            new IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>>(msg, fut));
+
+        assert old == null;
+
+        try {
+            io.apply(reduceAddrs[idx], msg);
+        }
+        catch (GridClosureException e) {
+            fut.onDone(U.unwrap(e));
+        }
+
+        fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> f) {
+                try {
+                    f.get();
+
+                    // Clean up the future from map only if there was no exception.
+                    // Otherwise flush() should fail.
+                    sentMsgs.remove(msgId);
+                }
+                catch (IgniteCheckedException e) {
+                    log.error("Failed to send message.", e);
+                }
+            }
+        });
+
+        msgs[idx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), idx,
+            Math.max(MSG_BUF_SIZE, newBufMinSize));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IgniteCheckedException {
+        if (snd != null) {
+            snd.cancel();
+
+            try {
+                snd.join();
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInterruptedCheckedException(e);
+            }
+        }
+
+        close(maps);
+    }
+
+    /**
+     * @param maps Maps.
+     */
+    private void close(AtomicReferenceArray<HadoopMultimap> maps) {
+        for (int i = 0; i < maps.length(); i++) {
+            HadoopMultimap map = maps.get(i);
+
+            if (map != null)
+                map.close();
+        }
+    }
+
+    /**
+     * @return Future.
+     */
+    @SuppressWarnings("unchecked")
+    public IgniteInternalFuture<?> flush() throws IgniteCheckedException {
+        if (log.isDebugEnabled())
+            log.debug("Flushing job " + job.id() + " on address " + locReduceAddr);
+
+        flushed = true;
+
+        if (maps.length() == 0)
+            return new GridFinishedFutureEx<>();
+
+        U.await(ioInitLatch);
+
+        GridWorker snd0 = snd;
+
+        if (snd0 != null) {
+            if (log.isDebugEnabled())
+                log.debug("Cancelling sender thread.");
+
+            snd0.cancel();
+
+            try {
+                snd0.join();
+
+                if (log.isDebugEnabled())
+                    log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInterruptedCheckedException(e);
+            }
+        }
+
+        collectUpdatesAndSend(true); // With flush.
+
+        if (log.isDebugEnabled())
+            log.debug("Finished sending collected updates to remote reducers: " + job.id());
+
+        GridCompoundFuture fut = new GridCompoundFuture<>();
+
+        for (IgniteBiTuple<HadoopShuffleMessage, GridFutureAdapterEx<?>> tup : sentMsgs.values())
+            fut.add(tup.get2());
+
+        fut.markInitialized();
+
+        if (log.isDebugEnabled())
+            log.debug("Collected futures to compound futures for flush: " + sentMsgs.size());
+
+        return fut;
+    }
+
+    /**
+     * @param taskCtx Task context.
+     * @return Output.
+     * @throws IgniteCheckedException If failed.
+     */
+    public GridHadoopTaskOutput output(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        switch (taskCtx.taskInfo().type()) {
+            case MAP:
+                assert !job.info().hasCombiner() : "The output creation is allowed if combiner has not been defined.";
+
+            case COMBINE:
+                return new PartitionedOutput(taskCtx);
+
+            default:
+                throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type());
+        }
+    }
+
+    /**
+     * @param taskCtx Task context.
+     * @return Input.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        switch (taskCtx.taskInfo().type()) {
+            case REDUCE:
+                int reducer = taskCtx.taskInfo().taskNumber();
+
+                HadoopMultimap m = maps.get(reducer);
+
+                if (m != null)
+                    return m.input(taskCtx);
+
+                return new GridHadoopTaskInput() { // Empty input.
+                    @Override public boolean next() {
+                        return false;
+                    }
+
+                    @Override public Object key() {
+                        throw new IllegalStateException();
+                    }
+
+                    @Override public Iterator<?> values() {
+                        throw new IllegalStateException();
+                    }
+
+                    @Override public void close() {
+                        // No-op.
+                    }
+                };
+
+            default:
+                throw new IllegalStateException("Illegal type: " + taskCtx.taskInfo().type());
+        }
+    }
+
+    /**
+     * Partitioned output.
+     */
+    private class PartitionedOutput implements GridHadoopTaskOutput {
+        /** */
+        private final GridHadoopTaskOutput[] adders = new GridHadoopTaskOutput[maps.length()];
+
+        /** */
+        private GridHadoopPartitioner partitioner;
+
+        /** */
+        private final GridHadoopTaskContext taskCtx;
+
+        /**
+         * Constructor.
+         * @param taskCtx Task context.
+         */
+        private PartitionedOutput(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+            this.taskCtx = taskCtx;
+
+            if (needPartitioner)
+                partitioner = taskCtx.partitioner();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object key, Object val) throws IgniteCheckedException {
+            int part = 0;
+
+            if (partitioner != null) {
+                part = partitioner.partition(key, val, adders.length);
+
+                if (part < 0 || part >= adders.length)
+                    throw new IgniteCheckedException("Invalid partition: " + part);
+            }
+
+            GridHadoopTaskOutput out = adders[part];
+
+            if (out == null)
+                adders[part] = out = getOrCreateMap(maps, part).startAdding(taskCtx);
+
+            out.write(key, val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            for (GridHadoopTaskOutput adder : adders) {
+                if (adder != null)
+                    adder.close();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
new file mode 100644
index 0000000..d227e75
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleMessage.java
@@ -0,0 +1,241 @@
+/*
+ * 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.shuffle;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory.*;
+
+/**
+ * Shuffle message.
+ */
+public class HadoopShuffleMessage implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final AtomicLong ids = new AtomicLong();
+
+    /** */
+    private static final byte MARKER_KEY = (byte)17;
+
+    /** */
+    private static final byte MARKER_VALUE = (byte)31;
+
+    /** */
+    @GridToStringInclude
+    private long msgId;
+
+    /** */
+    @GridToStringInclude
+    private GridHadoopJobId jobId;
+
+    /** */
+    @GridToStringInclude
+    private int reducer;
+
+    /** */
+    private byte[] buf;
+
+    /** */
+    @GridToStringInclude
+    private int off;
+
+    /**
+     *
+     */
+    public HadoopShuffleMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param size Size.
+     */
+    public HadoopShuffleMessage(GridHadoopJobId jobId, int reducer, int size) {
+        assert jobId != null;
+
+        buf = new byte[size];
+
+        this.jobId = jobId;
+        this.reducer = reducer;
+
+        msgId = ids.incrementAndGet();
+    }
+
+    /**
+     * @return Message ID.
+     */
+    public long id() {
+        return msgId;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @return Reducer.
+     */
+    public int reducer() {
+        return reducer;
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public byte[] buffer() {
+        return buf;
+    }
+
+    /**
+     * @return Offset.
+     */
+    public int offset() {
+        return off;
+    }
+
+    /**
+     * @param size Size.
+     * @param valOnly Only value wll be added.
+     * @return {@code true} If this message can fit additional data of this size
+     */
+    public boolean available(int size, boolean valOnly) {
+        size += valOnly ? 5 : 10;
+
+        if (off + size > buf.length) {
+            if (off == 0) { // Resize if requested size is too big.
+                buf = new byte[size];
+
+                return true;
+            }
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * @param keyPtr Key pointer.
+     * @param keySize Key size.
+     */
+    public void addKey(long keyPtr, int keySize) {
+        add(MARKER_KEY, keyPtr, keySize);
+    }
+
+    /**
+     * @param valPtr Value pointer.
+     * @param valSize Value size.
+     */
+    public void addValue(long valPtr, int valSize) {
+        add(MARKER_VALUE, valPtr, valSize);
+    }
+
+    /**
+     * @param marker Marker.
+     * @param ptr Pointer.
+     * @param size Size.
+     */
+    private void add(byte marker, long ptr, int size) {
+        buf[off++] = marker;
+
+        UNSAFE.putInt(buf, BYTE_ARR_OFF + off, size);
+
+        off += 4;
+
+        UNSAFE.copyMemory(null, ptr, buf, BYTE_ARR_OFF + off, size);
+
+        off += size;
+    }
+
+    /**
+     * @param v Visitor.
+     */
+    public void visit(Visitor v) throws IgniteCheckedException {
+        for (int i = 0; i < off;) {
+            byte marker = buf[i++];
+
+            int size = UNSAFE.getInt(buf, BYTE_ARR_OFF + i);
+
+            i += 4;
+
+            if (marker == MARKER_VALUE)
+                v.onValue(buf, i, size);
+            else if (marker == MARKER_KEY)
+                v.onKey(buf, i, size);
+            else
+                throw new IllegalStateException();
+
+            i += size;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+        out.writeLong(msgId);
+        out.writeInt(reducer);
+        out.writeInt(off);
+        U.writeByteArray(out, buf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new GridHadoopJobId();
+
+        jobId.readExternal(in);
+        msgId = in.readLong();
+        reducer = in.readInt();
+        off = in.readInt();
+        buf = U.readByteArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopShuffleMessage.class, this);
+    }
+
+    /**
+     * Visitor.
+     */
+    public static interface Visitor {
+        /**
+         * @param buf Buffer.
+         * @param off Offset.
+         * @param len Length.
+         */
+        public void onKey(byte[] buf, int off, int len) throws IgniteCheckedException;
+
+        /**
+         * @param buf Buffer.
+         * @param off Offset.
+         * @param len Length.
+         */
+        public void onValue(byte[] buf, int off, int len) throws IgniteCheckedException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java
deleted file mode 100644
index 32db722..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimap.java
+++ /dev/null
@@ -1,611 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Multimap for map reduce intermediate results.
- */
-public class GridHadoopConcurrentHashMultimap extends GridHadoopHashMultimapBase {
-    /** */
-    private final AtomicReference<State> state = new AtomicReference<>(State.READING_WRITING);
-
-    /** */
-    private volatile AtomicLongArray oldTbl;
-
-    /** */
-    private volatile AtomicLongArray newTbl;
-
-    /** */
-    private final AtomicInteger keys = new AtomicInteger();
-
-    /** */
-    private final CopyOnWriteArrayList<AdderImpl> adders = new CopyOnWriteArrayList<>();
-
-    /** */
-    private final AtomicInteger inputs = new AtomicInteger();
-
-    /**
-     * @param jobInfo Job info.
-     * @param mem Memory.
-     * @param cap Initial capacity.
-     */
-    public GridHadoopConcurrentHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) {
-        super(jobInfo, mem);
-
-        assert U.isPow2(cap);
-
-        newTbl = oldTbl = new AtomicLongArray(cap);
-    }
-
-    /**
-     * @return Number of keys.
-     */
-    public long keys() {
-        int res = keys.get();
-
-        for (AdderImpl adder : adders)
-            res += adder.locKeys.get();
-
-        return res;
-    }
-
-    /**
-     * @return Current table capacity.
-     */
-    @Override public int capacity() {
-        return oldTbl.length();
-    }
-
-    /**
-     * @return Adder object.
-     * @param ctx Task context.
-     */
-    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-        if (inputs.get() != 0)
-            throw new IllegalStateException("Active inputs.");
-
-        if (state.get() == State.CLOSING)
-            throw new IllegalStateException("Closed.");
-
-        return new AdderImpl(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        assert inputs.get() == 0 : inputs.get();
-        assert adders.isEmpty() : adders.size();
-
-        state(State.READING_WRITING, State.CLOSING);
-
-        if (keys() == 0)
-            return;
-
-        super.close();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected long meta(int idx) {
-        return oldTbl.get(idx);
-    }
-
-    /**
-     * Incrementally visits all the keys and values in the map.
-     *
-     * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning.
-     * @param v Visitor.
-     * @return {@code false} If visiting was impossible due to rehashing.
-     */
-    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
-        if (!state.compareAndSet(State.READING_WRITING, State.VISITING)) {
-            assert state.get() != State.CLOSING;
-
-            return false; // Can not visit while rehashing happens.
-        }
-
-        AtomicLongArray tbl0 = oldTbl;
-
-        for (int i = 0; i < tbl0.length(); i++) {
-            long meta = tbl0.get(i);
-
-            while (meta != 0) {
-                long valPtr = value(meta);
-
-                long lastVisited = ignoreLastVisited ? 0 : lastVisitedValue(meta);
-
-                if (valPtr != lastVisited) {
-                    v.onKey(key(meta), keySize(meta));
-
-                    lastVisitedValue(meta, valPtr); // Set it to the first value in chain.
-
-                    do {
-                        v.onValue(valPtr + 12, valueSize(valPtr));
-
-                        valPtr = nextValue(valPtr);
-                    }
-                    while (valPtr != lastVisited);
-                }
-
-                meta = collision(meta);
-            }
-        }
-
-        state(State.VISITING, State.READING_WRITING);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        inputs.incrementAndGet();
-
-        if (!adders.isEmpty())
-            throw new IllegalStateException("Active adders.");
-
-        State s = state.get();
-
-        if (s == State.CLOSING)
-            throw new IllegalStateException("Closed.");
-
-        assert s != State.REHASHING;
-
-        return new Input(taskCtx) {
-            @Override public void close() throws IgniteCheckedException {
-                if (inputs.decrementAndGet() < 0)
-                    throw new IllegalStateException();
-
-                super.close();
-            }
-        };
-    }
-
-    /**
-     * @param fromTbl Table.
-     */
-    private void rehashIfNeeded(AtomicLongArray fromTbl) {
-        if (fromTbl.length() == Integer.MAX_VALUE)
-            return;
-
-        long keys0 = keys();
-
-        if (keys0 < 3 * (fromTbl.length() >>> 2)) // New size has to be >= than 3/4 of capacity to rehash.
-            return;
-
-        if (fromTbl != newTbl) // Check if someone else have done the job.
-            return;
-
-        if (!state.compareAndSet(State.READING_WRITING, State.REHASHING)) {
-            assert state.get() != State.CLOSING; // Visiting is allowed, but we will not rehash.
-
-            return;
-        }
-
-        if (fromTbl != newTbl) { // Double check.
-            state(State.REHASHING, State.READING_WRITING); // Switch back.
-
-            return;
-        }
-
-        // Calculate new table capacity.
-        int newLen = fromTbl.length();
-
-        do {
-            newLen <<= 1;
-        }
-        while (newLen < keys0);
-
-        if (keys0 >= 3 * (newLen >>> 2)) // Still more than 3/4.
-            newLen <<= 1;
-
-        // This is our target table for rehashing.
-        AtomicLongArray toTbl = new AtomicLongArray(newLen);
-
-        // Make the new table visible before rehashing.
-        newTbl = toTbl;
-
-        // Rehash.
-        int newMask = newLen - 1;
-
-        long failedMeta = 0;
-
-        GridLongList collisions = new GridLongList(16);
-
-        for (int i = 0; i < fromTbl.length(); i++) { // Scan source table.
-            long meta = fromTbl.get(i);
-
-            assert meta != -1;
-
-            if (meta == 0) { // No entry.
-                failedMeta = 0;
-
-                if (!fromTbl.compareAndSet(i, 0, -1)) // Mark as moved.
-                    i--; // Retry.
-
-                continue;
-            }
-
-            do { // Collect all the collisions before the last one failed to nullify or 0.
-                collisions.add(meta);
-
-                meta = collision(meta);
-            }
-            while (meta != failedMeta);
-
-            do { // Go from the last to the first to avoid 'in-flight' state for meta entries.
-                meta = collisions.remove();
-
-                int addr = keyHash(meta) & newMask;
-
-                for (;;) { // Move meta entry to the new table.
-                    long toCollision = toTbl.get(addr);
-
-                    collision(meta, toCollision);
-
-                    if (toTbl.compareAndSet(addr, toCollision, meta))
-                        break;
-                }
-            }
-            while (!collisions.isEmpty());
-
-            // Here 'meta' will be a root pointer in old table.
-            if (!fromTbl.compareAndSet(i, meta, -1)) { // Try to mark as moved.
-                failedMeta = meta;
-
-                i--; // Retry the same address in table because new keys were added.
-            }
-            else
-                failedMeta = 0;
-        }
-
-        // Now old and new tables will be the same again.
-        oldTbl = toTbl;
-
-        state(State.REHASHING, State.READING_WRITING);
-    }
-
-    /**
-     * Switch state.
-     *
-     * @param oldState Expected state.
-     * @param newState New state.
-     */
-    private void state(State oldState, State newState) {
-        if (!state.compareAndSet(oldState, newState))
-            throw new IllegalStateException();
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Value pointer.
-     */
-    @Override protected long value(long meta) {
-        return mem.readLongVolatile(meta + 16);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param oldValPtr Old value.
-     * @param newValPtr New value.
-     * @return {@code true} If succeeded.
-     */
-    private boolean casValue(long meta, long oldValPtr, long newValPtr) {
-        return mem.casLong(meta + 16, oldValPtr, newValPtr);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Collision pointer.
-     */
-    @Override protected long collision(long meta) {
-        return mem.readLongVolatile(meta + 24);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param collision Collision pointer.
-     */
-    @Override protected void collision(long meta, long collision) {
-        assert meta != collision : meta;
-
-        mem.writeLongVolatile(meta + 24, collision);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Last visited value pointer.
-     */
-    private long lastVisitedValue(long meta) {
-        return mem.readLong(meta + 32);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param valPtr Last visited value pointer.
-     */
-    private void lastVisitedValue(long meta, long valPtr) {
-        mem.writeLong(meta + 32, valPtr);
-    }
-
-    /**
-     * Adder. Must not be shared between threads.
-     */
-    private class AdderImpl extends AdderBase {
-        /** */
-        private final Reader keyReader;
-
-        /** */
-        private final AtomicInteger locKeys = new AtomicInteger();
-
-        /** */
-        private final Random rnd = new GridRandom();
-
-        /**
-         * @param ctx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        private AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-            super(ctx);
-
-            keyReader = new Reader(keySer);
-
-            rehashIfNeeded(oldTbl);
-
-            adders.add(this);
-        }
-
-        /**
-         * @param in Data input.
-         * @param reuse Reusable key.
-         * @return Key.
-         * @throws IgniteCheckedException If failed.
-         */
-        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
-            KeyImpl k = reuse == null ? new KeyImpl() : (KeyImpl)reuse;
-
-            k.tmpKey = keySer.read(in, k.tmpKey);
-
-            k.meta = add(k.tmpKey, null);
-
-            return k;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Object key, Object val) throws IgniteCheckedException {
-            A.notNull(val, "val");
-
-            add(key, val);
-        }
-
-        /**
-         * @param tbl Table.
-         */
-        private void incrementKeys(AtomicLongArray tbl) {
-            locKeys.lazySet(locKeys.get() + 1);
-
-            if (rnd.nextInt(tbl.length()) < 512)
-                rehashIfNeeded(tbl);
-        }
-
-        /**
-         * @param keyHash Key hash.
-         * @param keySize Key size.
-         * @param keyPtr Key pointer.
-         * @param valPtr Value page pointer.
-         * @param collisionPtr Pointer to meta with hash collision.
-         * @param lastVisitedVal Last visited value pointer.
-         * @return Created meta page pointer.
-         */
-        private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr, long lastVisitedVal) {
-            long meta = allocate(40);
-
-            mem.writeInt(meta, keyHash);
-            mem.writeInt(meta + 4, keySize);
-            mem.writeLong(meta + 8, keyPtr);
-            mem.writeLong(meta + 16, valPtr);
-            mem.writeLong(meta + 24, collisionPtr);
-            mem.writeLong(meta + 32, lastVisitedVal);
-
-            return meta;
-        }
-
-        /**
-         * @param key Key.
-         * @param val Value.
-         * @return Updated or created meta page pointer.
-         * @throws IgniteCheckedException If failed.
-         */
-        private long add(Object key, @Nullable Object val) throws IgniteCheckedException {
-            AtomicLongArray tbl = oldTbl;
-
-            int keyHash = U.hash(key.hashCode());
-
-            long newMetaPtr = 0;
-
-            long valPtr = 0;
-
-            if (val != null) {
-                valPtr = write(12, val, valSer);
-                int valSize = writtenSize() - 12;
-
-                valueSize(valPtr, valSize);
-            }
-
-            for (AtomicLongArray old = null;;) {
-                int addr = keyHash & (tbl.length() - 1);
-
-                long metaPtrRoot = tbl.get(addr); // Read root meta pointer at this address.
-
-                if (metaPtrRoot == -1) { // The cell was already moved by rehashing.
-                    AtomicLongArray n = newTbl; // Need to read newTbl first here.
-                    AtomicLongArray o = oldTbl;
-
-                    tbl = tbl == o ? n : o; // Trying to get the oldest table but newer than ours.
-
-                    old = null;
-
-                    continue;
-                }
-
-                if (metaPtrRoot != 0) { // Not empty slot.
-                    long metaPtr = metaPtrRoot;
-
-                    do { // Scan all the collisions.
-                        if (keyHash(metaPtr) == keyHash && key.equals(keyReader.readKey(metaPtr))) { // Found key.
-                            if (newMetaPtr != 0)  // Deallocate new meta if one was allocated.
-                                localDeallocate(key(newMetaPtr)); // Key was allocated first, so rewind to it's pointer.
-
-                            if (valPtr != 0) { // Add value if it exists.
-                                long nextValPtr;
-
-                                // Values are linked to each other to a stack like structure.
-                                // Replace the last value in meta with ours and link it as next.
-                                do {
-                                    nextValPtr = value(metaPtr);
-
-                                    nextValue(valPtr, nextValPtr);
-                                }
-                                while (!casValue(metaPtr, nextValPtr, valPtr));
-                            }
-
-                            return metaPtr;
-                        }
-
-                        metaPtr = collision(metaPtr);
-                    }
-                    while (metaPtr != 0);
-
-                    // Here we did not find our key, need to check if it was moved by rehashing to the new table.
-                    if (old == null) { // If the old table already set, then we will just try to update it.
-                        AtomicLongArray n = newTbl;
-
-                        if (n != tbl) { // Rehashing happens, try to find the key in new table but preserve the old one.
-                            old = tbl;
-                            tbl = n;
-
-                            continue;
-                        }
-                    }
-                }
-
-                if (old != null) { // We just checked new table but did not find our key as well as in the old one.
-                    tbl = old; // Try to add new key to the old table.
-
-                    addr = keyHash & (tbl.length() - 1);
-
-                    old = null;
-                }
-
-                if (newMetaPtr == 0) { // Allocate new meta page.
-                    long keyPtr = write(0, key, keySer);
-                    int keySize = writtenSize();
-
-                    if (valPtr != 0)
-                        nextValue(valPtr, 0);
-
-                    newMetaPtr = createMeta(keyHash, keySize, keyPtr, valPtr, metaPtrRoot, 0);
-                }
-                else // Update new meta with root pointer collision.
-                    collision(newMetaPtr, metaPtrRoot);
-
-                if (tbl.compareAndSet(addr, metaPtrRoot, newMetaPtr)) { // Try to replace root pointer with new one.
-                    incrementKeys(tbl);
-
-                    return newMetaPtr;
-                }
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            if (!adders.remove(this))
-                throw new IllegalStateException();
-
-            keys.addAndGet(locKeys.get()); // Here we have race and #keys() method can return wrong result but it is ok.
-
-            super.close();
-        }
-
-        /**
-         * Key.
-         */
-        private class KeyImpl implements Key {
-            /** */
-            private long meta;
-
-            /** */
-            private Object tmpKey;
-
-            /**
-             * @return Meta pointer for the key.
-             */
-            public long address() {
-                return meta;
-            }
-
-            /**
-             * @param val Value.
-             */
-            @Override public void add(Value val) {
-                int size = val.size();
-
-                long valPtr = allocate(size + 12);
-
-                val.copyTo(valPtr + 12);
-
-                valueSize(valPtr, size);
-
-                long nextVal;
-
-                do {
-                    nextVal = value(meta);
-
-                    nextValue(valPtr, nextVal);
-                }
-                while(!casValue(meta, nextVal, valPtr));
-            }
-        }
-    }
-
-    /**
-     * Current map state.
-     */
-    private enum State {
-        /** */
-        REHASHING,
-
-        /** */
-        VISITING,
-
-        /** */
-        READING_WRITING,
-
-        /** */
-        CLOSING
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java
deleted file mode 100644
index 2795b77..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimap.java
+++ /dev/null
@@ -1,174 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-/**
- * Hash multimap.
- */
-public class GridHadoopHashMultimap extends GridHadoopHashMultimapBase {
-    /** */
-    private long[] tbl;
-
-    /** */
-    private int keys;
-
-    /**
-     * @param jobInfo Job info.
-     * @param mem Memory.
-     * @param cap Initial capacity.
-     */
-    public GridHadoopHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) {
-        super(jobInfo, mem);
-
-        assert U.isPow2(cap) : cap;
-
-        tbl = new long[cap];
-    }
-
-    /** {@inheritDoc} */
-    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-        return new AdderImpl(ctx);
-    }
-
-    /**
-     * Rehash.
-     */
-    private void rehash() {
-        long[] newTbl = new long[tbl.length << 1];
-
-        int newMask = newTbl.length - 1;
-
-        for (long meta : tbl) {
-            while (meta != 0) {
-                long collision = collision(meta);
-
-                int idx = keyHash(meta) & newMask;
-
-                collision(meta, newTbl[idx]);
-
-                newTbl[idx] = meta;
-
-                meta = collision;
-            }
-        }
-
-        tbl = newTbl;
-    }
-
-    /**
-     * @return Keys count.
-     */
-    public int keys() {
-        return keys;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int capacity() {
-        return tbl.length;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected long meta(int idx) {
-        return tbl[idx];
-    }
-
-    /**
-     * Adder.
-     */
-    private class AdderImpl extends AdderBase {
-        /** */
-        private final Reader keyReader;
-
-        /**
-         * @param ctx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-            super(ctx);
-
-            keyReader = new Reader(keySer);
-        }
-
-        /**
-         * @param keyHash Key hash.
-         * @param keySize Key size.
-         * @param keyPtr Key pointer.
-         * @param valPtr Value page pointer.
-         * @param collisionPtr Pointer to meta with hash collision.
-         * @return Created meta page pointer.
-         */
-        private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr) {
-            long meta = allocate(32);
-
-            mem.writeInt(meta, keyHash);
-            mem.writeInt(meta + 4, keySize);
-            mem.writeLong(meta + 8, keyPtr);
-            mem.writeLong(meta + 16, valPtr);
-            mem.writeLong(meta + 24, collisionPtr);
-
-            return meta;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Object key, Object val) throws IgniteCheckedException {
-            A.notNull(val, "val");
-
-            int keyHash = U.hash(key.hashCode());
-
-            // Write value.
-            long valPtr = write(12, val, valSer);
-            int valSize = writtenSize() - 12;
-
-            valueSize(valPtr, valSize);
-
-            // Find position in table.
-            int idx = keyHash & (tbl.length - 1);
-
-            long meta = tbl[idx];
-
-            // Search for our key in collisions.
-            while (meta != 0) {
-                if (keyHash(meta) == keyHash && key.equals(keyReader.readKey(meta))) { // Found key.
-                    nextValue(valPtr, value(meta));
-
-                    value(meta, valPtr);
-
-                    return;
-                }
-
-                meta = collision(meta);
-            }
-
-            // Write key.
-            long keyPtr = write(0, key, keySer);
-            int keySize = writtenSize();
-
-            nextValue(valPtr, 0);
-
-            tbl[idx] = createMeta(keyHash, keySize, keyPtr, valPtr, tbl[idx]);
-
-            if (++keys > (tbl.length >>> 2) * 3)
-                rehash();
-        }
-    }
-}


[14/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopFileSystemsUtils.java
deleted file mode 100644
index e1bf9b6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopFileSystemsUtils.java
+++ /dev/null
@@ -1,57 +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.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.ignite.hadoop.fs.v1.*;
-
-/**
- * Utilities for configuring file systems to support the separate working directory per each thread.
- */
-public class GridHadoopFileSystemsUtils {
-    /** Name of the property for setting working directory on create new local FS instance. */
-    public static final String LOC_FS_WORK_DIR_PROP = "fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".workDir";
-
-    /**
-     * Set user name and default working directory for current thread if it's supported by file system.
-     *
-     * @param fs File system.
-     * @param userName User name.
-     */
-    public static void setUser(FileSystem fs, String userName) {
-        if (fs instanceof IgniteHadoopFileSystem)
-            ((IgniteHadoopFileSystem)fs).setUser(userName);
-        else if (fs instanceof GridHadoopDistributedFileSystem)
-            ((GridHadoopDistributedFileSystem)fs).setUser(userName);
-    }
-
-    /**
-     * Setup wrappers of filesystems to support the separate working directory.
-     *
-     * @param cfg Config for setup.
-     */
-    public static void setupFileSystems(Configuration cfg) {
-        cfg.set("fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl", GridHadoopLocalFileSystemV1.class.getName());
-        cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
-                GridHadoopLocalFileSystemV2.class.getName());
-
-        cfg.set("fs." + HdfsConstants.HDFS_URI_SCHEME + ".impl", GridHadoopDistributedFileSystem.class.getName());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV1.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV1.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV1.java
deleted file mode 100644
index 28834d4..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV1.java
+++ /dev/null
@@ -1,39 +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.fs;
-
-import org.apache.hadoop.fs.*;
-
-import java.io.*;
-
-/**
- * Local file system replacement for Hadoop jobs.
- */
-public class GridHadoopLocalFileSystemV1 extends LocalFileSystem {
-    /**
-     * Creates new local file system.
-     */
-    public GridHadoopLocalFileSystemV1() {
-        super(new GridHadoopRawLocalFileSystem());
-    }
-
-    /** {@inheritDoc} */
-    @Override public File pathToFile(Path path) {
-        return ((GridHadoopRawLocalFileSystem)getRaw()).convert(path);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV2.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV2.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV2.java
deleted file mode 100644
index 62d7cea..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopLocalFileSystemV2.java
+++ /dev/null
@@ -1,86 +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.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.local.*;
-
-import java.io.*;
-import java.net.*;
-
-import static org.apache.hadoop.fs.FsConstants.*;
-
-/**
- * Local file system replacement for Hadoop jobs.
- */
-public class GridHadoopLocalFileSystemV2 extends ChecksumFs {
-    /**
-     * Creates new local file system.
-     *
-     * @param cfg Configuration.
-     * @throws IOException If failed.
-     * @throws URISyntaxException If failed.
-     */
-    public GridHadoopLocalFileSystemV2(Configuration cfg) throws IOException, URISyntaxException {
-        super(new DelegateFS(cfg));
-    }
-
-    /**
-     * Creates new local file system.
-     *
-     * @param uri URI.
-     * @param cfg Configuration.
-     * @throws IOException If failed.
-     * @throws URISyntaxException If failed.
-     */
-    public GridHadoopLocalFileSystemV2(URI uri, Configuration cfg) throws IOException, URISyntaxException {
-        this(cfg);
-    }
-
-    /**
-     * Delegate file system.
-     */
-    private static class DelegateFS extends DelegateToFileSystem {
-        /**
-         * Creates new local file system.
-         *
-         * @param cfg Configuration.
-         * @throws IOException If failed.
-         * @throws URISyntaxException If failed.
-         */
-        public DelegateFS(Configuration cfg) throws IOException, URISyntaxException {
-            super(LOCAL_FS_URI, new GridHadoopRawLocalFileSystem(), cfg, LOCAL_FS_URI.getScheme(), false);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int getUriDefaultPort() {
-            return -1;
-        }
-
-        /** {@inheritDoc} */
-        @Override public FsServerDefaults getServerDefaults() throws IOException {
-            return LocalConfigKeys.getServerDefaults();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isValidName(String src) {
-            return true;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopRawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopRawLocalFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopRawLocalFileSystem.java
deleted file mode 100644
index 29645f8..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopRawLocalFileSystem.java
+++ /dev/null
@@ -1,304 +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.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.file.*;
-
-/**
- * Local file system implementation for Hadoop.
- */
-public class GridHadoopRawLocalFileSystem extends FileSystem {
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workDir = new ThreadLocal<Path>() {
-        @Override protected Path initialValue() {
-            return getInitialWorkingDirectory();
-        }
-    };
-
-    /**
-     * Converts Hadoop path to local path.
-     *
-     * @param path Hadoop path.
-     * @return Local path.
-     */
-    File convert(Path path) {
-        checkPath(path);
-
-        if (path.isAbsolute())
-            return new File(path.toUri().getPath());
-
-        return new File(getWorkingDirectory().toUri().getPath(), path.toUri().getPath());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getHomeDirectory() {
-        return makeQualified(new Path(System.getProperty("user.home")));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getInitialWorkingDirectory() {
-        File f = new File(System.getProperty("user.dir"));
-
-        return new Path(f.getAbsoluteFile().toURI()).makeQualified(getUri(), null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void initialize(URI uri, Configuration conf) throws IOException {
-        super.initialize(uri, conf);
-
-        setConf(conf);
-
-        String initWorkDir = conf.get(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP);
-
-        if (initWorkDir != null)
-            setWorkingDirectory(new Path(initWorkDir));
-    }
-
-    /** {@inheritDoc} */
-    @Override public URI getUri() {
-        return FsConstants.LOCAL_FS_URI;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException {
-        return new FSDataInputStream(new InStream(checkExists(convert(f))));
-    }
-
-    /** {@inheritDoc} */
-    @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufSize,
-        short replication, long blockSize, Progressable progress) throws IOException {
-        File file = convert(f);
-
-        if (!overwrite && !file.createNewFile())
-            throw new IOException("Failed to create new file: " + f.toUri());
-
-        return out(file, false, bufSize);
-    }
-
-    /**
-     * @param file File.
-     * @param append Append flag.
-     * @return Output stream.
-     * @throws IOException If failed.
-     */
-    private FSDataOutputStream out(File file, boolean append, int bufSize) throws IOException {
-        return new FSDataOutputStream(new BufferedOutputStream(new FileOutputStream(file, append),
-            bufSize < 32 * 1024 ? 32 * 1024 : bufSize), new Statistics(getUri().getScheme()));
-    }
-
-    /** {@inheritDoc} */
-    @Override public FSDataOutputStream append(Path f, int bufSize, Progressable progress) throws IOException {
-        return out(convert(f), true, bufSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean rename(Path src, Path dst) throws IOException {
-        return convert(src).renameTo(convert(dst));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean delete(Path f, boolean recursive) throws IOException {
-        File file = convert(f);
-
-        if (file.isDirectory() && !recursive)
-            throw new IOException("Failed to remove directory in non recursive mode: " + f.toUri());
-
-        return U.delete(file);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setWorkingDirectory(Path dir) {
-        workDir.set(fixRelativePart(dir));
-
-        checkPath(dir);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getWorkingDirectory() {
-        return workDir.get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException {
-        if(f == null)
-            throw new IllegalArgumentException("mkdirs path arg is null");
-
-        Path parent = f.getParent();
-
-        File p2f = convert(f);
-
-        if(parent != null) {
-            File parent2f = convert(parent);
-
-            if(parent2f != null && parent2f.exists() && !parent2f.isDirectory())
-                throw new FileAlreadyExistsException("Parent path is not a directory: " + parent);
-
-        }
-
-        return (parent == null || mkdirs(parent)) && (p2f.mkdir() || p2f.isDirectory());
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus getFileStatus(Path f) throws IOException {
-        return fileStatus(checkExists(convert(f)));
-    }
-
-    /**
-     * @return File status.
-     */
-    private FileStatus fileStatus(File file) throws IOException {
-        boolean dir = file.isDirectory();
-
-        java.nio.file.Path path = dir ? null : file.toPath();
-
-        return new FileStatus(dir ? 0 : file.length(), dir, 1, 4 * 1024, file.lastModified(), file.lastModified(),
-            /*permission*/null, /*owner*/null, /*group*/null, dir ? null : Files.isSymbolicLink(path) ?
-            new Path(Files.readSymbolicLink(path).toUri()) : null, new Path(file.toURI()));
-    }
-
-    /**
-     * @param file File.
-     * @return Same file.
-     * @throws FileNotFoundException If does not exist.
-     */
-    private static File checkExists(File file) throws FileNotFoundException {
-        if (!file.exists())
-            throw new FileNotFoundException("File " + file.getAbsolutePath() + " does not exist.");
-
-        return file;
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus[] listStatus(Path f) throws IOException {
-        File file = convert(f);
-
-        if (checkExists(file).isFile())
-            return new FileStatus[] {fileStatus(file)};
-
-        File[] files = file.listFiles();
-
-        FileStatus[] res = new FileStatus[files.length];
-
-        for (int i = 0; i < res.length; i++)
-            res[i] = fileStatus(files[i]);
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean supportsSymlinks() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
-        Files.createSymbolicLink(convert(link).toPath(), convert(target).toPath());
-    }
-
-    /** {@inheritDoc} */
-    @Override public FileStatus getFileLinkStatus(Path f) throws IOException {
-        return getFileStatus(getLinkTarget(f));
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getLinkTarget(Path f) throws IOException {
-        File file = Files.readSymbolicLink(convert(f).toPath()).toFile();
-
-        return new Path(file.toURI());
-    }
-
-    /**
-     * Input stream.
-     */
-    private static class InStream extends InputStream implements Seekable, PositionedReadable {
-        /** */
-        private final RandomAccessFile file;
-
-        /**
-         * @param f File.
-         * @throws IOException If failed.
-         */
-        public InStream(File f) throws IOException {
-            file = new RandomAccessFile(f, "r");
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized int read() throws IOException {
-            return file.read();
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized int read(byte[] b, int off, int len) throws IOException {
-            return file.read(b, off, len);
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized void close() throws IOException {
-            file.close();
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized int read(long pos, byte[] buf, int off, int len) throws IOException {
-            long pos0 = file.getFilePointer();
-
-            file.seek(pos);
-            int res = file.read(buf, off, len);
-
-            file.seek(pos0);
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readFully(long pos, byte[] buf, int off, int len) throws IOException {
-            if (read(pos, buf, off, len) != len)
-                throw new IOException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void readFully(long pos, byte[] buf) throws IOException {
-            readFully(pos, buf, 0, buf.length);
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized void seek(long pos) throws IOException {
-            file.seek(pos);
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized long getPos() throws IOException {
-            return file.getFilePointer();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean seekToNewSource(long targetPos) throws IOException {
-            return false;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
new file mode 100644
index 0000000..88c5899
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
@@ -0,0 +1,91 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.*;
+import org.apache.hadoop.mapreduce.*;
+
+import java.io.*;
+import java.net.*;
+
+import static org.apache.ignite.configuration.IgfsConfiguration.*;
+
+/**
+ * Wrapper of HDFS for support of separated working directory.
+ */
+public class HadoopDistributedFileSystem extends DistributedFileSystem {
+    /** User name for each thread. */
+    private final ThreadLocal<String> userName = new ThreadLocal<String>() {
+        /** {@inheritDoc} */
+        @Override protected String initialValue() {
+            return DFLT_USER_NAME;
+        }
+    };
+
+    /** Working directory for each thread. */
+    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>() {
+        /** {@inheritDoc} */
+        @Override protected Path initialValue() {
+            return getHomeDirectory();
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override public void initialize(URI uri, Configuration conf) throws IOException {
+        super.initialize(uri, conf);
+
+        setUser(conf.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+    }
+
+    /**
+     * Set user name and default working directory for current thread.
+     *
+     * @param userName User name.
+     */
+    public void setUser(String userName) {
+        this.userName.set(userName);
+
+        setWorkingDirectory(getHomeDirectory());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getHomeDirectory() {
+        Path path = new Path("/user/" + userName.get());
+
+        return path.makeQualified(getUri(), null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setWorkingDirectory(Path dir) {
+        Path fixedDir = fixRelativePart(dir);
+
+        String res = fixedDir.toUri().getPath();
+
+        if (!DFSUtil.isValidName(res))
+            throw new IllegalArgumentException("Invalid DFS directory name " + res);
+
+        workingDir.set(fixedDir);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getWorkingDirectory() {
+        return workingDir.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
new file mode 100644
index 0000000..f3f51d4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
@@ -0,0 +1,57 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.hdfs.protocol.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+
+/**
+ * Utilities for configuring file systems to support the separate working directory per each thread.
+ */
+public class HadoopFileSystemsUtils {
+    /** Name of the property for setting working directory on create new local FS instance. */
+    public static final String LOC_FS_WORK_DIR_PROP = "fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".workDir";
+
+    /**
+     * Set user name and default working directory for current thread if it's supported by file system.
+     *
+     * @param fs File system.
+     * @param userName User name.
+     */
+    public static void setUser(FileSystem fs, String userName) {
+        if (fs instanceof IgniteHadoopFileSystem)
+            ((IgniteHadoopFileSystem)fs).setUser(userName);
+        else if (fs instanceof HadoopDistributedFileSystem)
+            ((HadoopDistributedFileSystem)fs).setUser(userName);
+    }
+
+    /**
+     * Setup wrappers of filesystems to support the separate working directory.
+     *
+     * @param cfg Config for setup.
+     */
+    public static void setupFileSystems(Configuration cfg) {
+        cfg.set("fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl", HadoopLocalFileSystemV1.class.getName());
+        cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
+                HadoopLocalFileSystemV2.class.getName());
+
+        cfg.set("fs." + HdfsConstants.HDFS_URI_SCHEME + ".impl", HadoopDistributedFileSystem.class.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java
new file mode 100644
index 0000000..9cc5881
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV1.java
@@ -0,0 +1,39 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.fs.*;
+
+import java.io.*;
+
+/**
+ * Local file system replacement for Hadoop jobs.
+ */
+public class HadoopLocalFileSystemV1 extends LocalFileSystem {
+    /**
+     * Creates new local file system.
+     */
+    public HadoopLocalFileSystemV1() {
+        super(new HadoopRawLocalFileSystem());
+    }
+
+    /** {@inheritDoc} */
+    @Override public File pathToFile(Path path) {
+        return ((HadoopRawLocalFileSystem)getRaw()).convert(path);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java
new file mode 100644
index 0000000..15ddc5a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLocalFileSystemV2.java
@@ -0,0 +1,86 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.local.*;
+
+import java.io.*;
+import java.net.*;
+
+import static org.apache.hadoop.fs.FsConstants.*;
+
+/**
+ * Local file system replacement for Hadoop jobs.
+ */
+public class HadoopLocalFileSystemV2 extends ChecksumFs {
+    /**
+     * Creates new local file system.
+     *
+     * @param cfg Configuration.
+     * @throws IOException If failed.
+     * @throws URISyntaxException If failed.
+     */
+    public HadoopLocalFileSystemV2(Configuration cfg) throws IOException, URISyntaxException {
+        super(new DelegateFS(cfg));
+    }
+
+    /**
+     * Creates new local file system.
+     *
+     * @param uri URI.
+     * @param cfg Configuration.
+     * @throws IOException If failed.
+     * @throws URISyntaxException If failed.
+     */
+    public HadoopLocalFileSystemV2(URI uri, Configuration cfg) throws IOException, URISyntaxException {
+        this(cfg);
+    }
+
+    /**
+     * Delegate file system.
+     */
+    private static class DelegateFS extends DelegateToFileSystem {
+        /**
+         * Creates new local file system.
+         *
+         * @param cfg Configuration.
+         * @throws IOException If failed.
+         * @throws URISyntaxException If failed.
+         */
+        public DelegateFS(Configuration cfg) throws IOException, URISyntaxException {
+            super(LOCAL_FS_URI, new HadoopRawLocalFileSystem(), cfg, LOCAL_FS_URI.getScheme(), false);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int getUriDefaultPort() {
+            return -1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public FsServerDefaults getServerDefaults() throws IOException {
+            return LocalConfigKeys.getServerDefaults();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isValidName(String src) {
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java
new file mode 100644
index 0000000..e5ec3f7
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopRawLocalFileSystem.java
@@ -0,0 +1,304 @@
+/*
+ * 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.fs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.util.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.net.*;
+import java.nio.file.*;
+
+/**
+ * Local file system implementation for Hadoop.
+ */
+public class HadoopRawLocalFileSystem extends FileSystem {
+    /** Working directory for each thread. */
+    private final ThreadLocal<Path> workDir = new ThreadLocal<Path>() {
+        @Override protected Path initialValue() {
+            return getInitialWorkingDirectory();
+        }
+    };
+
+    /**
+     * Converts Hadoop path to local path.
+     *
+     * @param path Hadoop path.
+     * @return Local path.
+     */
+    File convert(Path path) {
+        checkPath(path);
+
+        if (path.isAbsolute())
+            return new File(path.toUri().getPath());
+
+        return new File(getWorkingDirectory().toUri().getPath(), path.toUri().getPath());
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getHomeDirectory() {
+        return makeQualified(new Path(System.getProperty("user.home")));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getInitialWorkingDirectory() {
+        File f = new File(System.getProperty("user.dir"));
+
+        return new Path(f.getAbsoluteFile().toURI()).makeQualified(getUri(), null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void initialize(URI uri, Configuration conf) throws IOException {
+        super.initialize(uri, conf);
+
+        setConf(conf);
+
+        String initWorkDir = conf.get(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP);
+
+        if (initWorkDir != null)
+            setWorkingDirectory(new Path(initWorkDir));
+    }
+
+    /** {@inheritDoc} */
+    @Override public URI getUri() {
+        return FsConstants.LOCAL_FS_URI;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+        return new FSDataInputStream(new InStream(checkExists(convert(f))));
+    }
+
+    /** {@inheritDoc} */
+    @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufSize,
+        short replication, long blockSize, Progressable progress) throws IOException {
+        File file = convert(f);
+
+        if (!overwrite && !file.createNewFile())
+            throw new IOException("Failed to create new file: " + f.toUri());
+
+        return out(file, false, bufSize);
+    }
+
+    /**
+     * @param file File.
+     * @param append Append flag.
+     * @return Output stream.
+     * @throws IOException If failed.
+     */
+    private FSDataOutputStream out(File file, boolean append, int bufSize) throws IOException {
+        return new FSDataOutputStream(new BufferedOutputStream(new FileOutputStream(file, append),
+            bufSize < 32 * 1024 ? 32 * 1024 : bufSize), new Statistics(getUri().getScheme()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public FSDataOutputStream append(Path f, int bufSize, Progressable progress) throws IOException {
+        return out(convert(f), true, bufSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean rename(Path src, Path dst) throws IOException {
+        return convert(src).renameTo(convert(dst));
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean delete(Path f, boolean recursive) throws IOException {
+        File file = convert(f);
+
+        if (file.isDirectory() && !recursive)
+            throw new IOException("Failed to remove directory in non recursive mode: " + f.toUri());
+
+        return U.delete(file);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setWorkingDirectory(Path dir) {
+        workDir.set(fixRelativePart(dir));
+
+        checkPath(dir);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getWorkingDirectory() {
+        return workDir.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+        if(f == null)
+            throw new IllegalArgumentException("mkdirs path arg is null");
+
+        Path parent = f.getParent();
+
+        File p2f = convert(f);
+
+        if(parent != null) {
+            File parent2f = convert(parent);
+
+            if(parent2f != null && parent2f.exists() && !parent2f.isDirectory())
+                throw new FileAlreadyExistsException("Parent path is not a directory: " + parent);
+
+        }
+
+        return (parent == null || mkdirs(parent)) && (p2f.mkdir() || p2f.isDirectory());
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus getFileStatus(Path f) throws IOException {
+        return fileStatus(checkExists(convert(f)));
+    }
+
+    /**
+     * @return File status.
+     */
+    private FileStatus fileStatus(File file) throws IOException {
+        boolean dir = file.isDirectory();
+
+        java.nio.file.Path path = dir ? null : file.toPath();
+
+        return new FileStatus(dir ? 0 : file.length(), dir, 1, 4 * 1024, file.lastModified(), file.lastModified(),
+            /*permission*/null, /*owner*/null, /*group*/null, dir ? null : Files.isSymbolicLink(path) ?
+            new Path(Files.readSymbolicLink(path).toUri()) : null, new Path(file.toURI()));
+    }
+
+    /**
+     * @param file File.
+     * @return Same file.
+     * @throws FileNotFoundException If does not exist.
+     */
+    private static File checkExists(File file) throws FileNotFoundException {
+        if (!file.exists())
+            throw new FileNotFoundException("File " + file.getAbsolutePath() + " does not exist.");
+
+        return file;
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus[] listStatus(Path f) throws IOException {
+        File file = convert(f);
+
+        if (checkExists(file).isFile())
+            return new FileStatus[] {fileStatus(file)};
+
+        File[] files = file.listFiles();
+
+        FileStatus[] res = new FileStatus[files.length];
+
+        for (int i = 0; i < res.length; i++)
+            res[i] = fileStatus(files[i]);
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean supportsSymlinks() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void createSymlink(Path target, Path link, boolean createParent) throws IOException {
+        Files.createSymbolicLink(convert(link).toPath(), convert(target).toPath());
+    }
+
+    /** {@inheritDoc} */
+    @Override public FileStatus getFileLinkStatus(Path f) throws IOException {
+        return getFileStatus(getLinkTarget(f));
+    }
+
+    /** {@inheritDoc} */
+    @Override public Path getLinkTarget(Path f) throws IOException {
+        File file = Files.readSymbolicLink(convert(f).toPath()).toFile();
+
+        return new Path(file.toURI());
+    }
+
+    /**
+     * Input stream.
+     */
+    private static class InStream extends InputStream implements Seekable, PositionedReadable {
+        /** */
+        private final RandomAccessFile file;
+
+        /**
+         * @param f File.
+         * @throws IOException If failed.
+         */
+        public InStream(File f) throws IOException {
+            file = new RandomAccessFile(f, "r");
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized int read() throws IOException {
+            return file.read();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized int read(byte[] b, int off, int len) throws IOException {
+            return file.read(b, off, len);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void close() throws IOException {
+            file.close();
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized int read(long pos, byte[] buf, int off, int len) throws IOException {
+            long pos0 = file.getFilePointer();
+
+            file.seek(pos);
+            int res = file.read(buf, off, len);
+
+            file.seek(pos0);
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readFully(long pos, byte[] buf, int off, int len) throws IOException {
+            if (read(pos, buf, off, len) != len)
+                throw new IOException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readFully(long pos, byte[] buf) throws IOException {
+            readFully(pos, buf, 0, buf.length);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void seek(long pos) throws IOException {
+            file.seek(pos);
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized long getPos() throws IOException {
+            return file.getFilePointer();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean seekToNewSource(long targetPos) throws IOException {
+            return false;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobMetadata.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobMetadata.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobMetadata.java
deleted file mode 100644
index b124312..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobMetadata.java
+++ /dev/null
@@ -1,305 +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.jobtracker;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
-
-/**
- * Hadoop job metadata. Internal object used for distributed job state tracking.
- */
-public class GridHadoopJobMetadata implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Job ID. */
-    private GridHadoopJobId jobId;
-
-    /** Job info. */
-    private GridHadoopJobInfo jobInfo;
-
-    /** Node submitted job. */
-    private UUID submitNodeId;
-
-    /** Map-reduce plan. */
-    private GridHadoopMapReducePlan mrPlan;
-
-    /** Pending splits for which mapper should be executed. */
-    private Map<GridHadoopInputSplit, Integer> pendingSplits;
-
-    /** Pending reducers. */
-    private Collection<Integer> pendingReducers;
-
-    /** Reducers addresses. */
-    @GridToStringInclude
-    private Map<Integer, GridHadoopProcessDescriptor> reducersAddrs;
-
-    /** Job phase. */
-    private GridHadoopJobPhase phase = PHASE_SETUP;
-
-    /** Fail cause. */
-    @GridToStringExclude
-    private Throwable failCause;
-
-    /** Version. */
-    private long ver;
-
-    /** Job counters */
-    private GridHadoopCounters counters = new GridHadoopCountersImpl();
-
-    /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridHadoopJobMetadata() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param submitNodeId Submit node ID.
-     * @param jobId Job ID.
-     * @param jobInfo Job info.
-     */
-    public GridHadoopJobMetadata(UUID submitNodeId, GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
-        this.jobId = jobId;
-        this.jobInfo = jobInfo;
-        this.submitNodeId = submitNodeId;
-    }
-
-    /**
-     * Copy constructor.
-     *
-     * @param src Metadata to copy.
-     */
-    public GridHadoopJobMetadata(GridHadoopJobMetadata src) {
-        // Make sure to preserve alphabetic order.
-        counters = src.counters;
-        failCause = src.failCause;
-        jobId = src.jobId;
-        jobInfo = src.jobInfo;
-        mrPlan = src.mrPlan;
-        pendingSplits = src.pendingSplits;
-        pendingReducers = src.pendingReducers;
-        phase = src.phase;
-        reducersAddrs = src.reducersAddrs;
-        submitNodeId = src.submitNodeId;
-        ver = src.ver + 1;
-    }
-
-    /**
-     * @return Submit node ID.
-     */
-    public UUID submitNodeId() {
-        return submitNodeId;
-    }
-
-    /**
-     * @param phase Job phase.
-     */
-    public void phase(GridHadoopJobPhase phase) {
-        this.phase = phase;
-    }
-
-    /**
-     * @return Job phase.
-     */
-    public GridHadoopJobPhase phase() {
-        return phase;
-    }
-
-    /**
-     * Gets reducers addresses for external execution.
-     *
-     * @return Reducers addresses.
-     */
-    public Map<Integer, GridHadoopProcessDescriptor> reducersAddresses() {
-        return reducersAddrs;
-    }
-
-    /**
-     * Sets reducers addresses for external execution.
-     *
-     * @param reducersAddrs Map of addresses.
-     */
-    public void reducersAddresses(Map<Integer, GridHadoopProcessDescriptor> reducersAddrs) {
-        this.reducersAddrs = reducersAddrs;
-    }
-
-    /**
-     * Sets collection of pending splits.
-     *
-     * @param pendingSplits Collection of pending splits.
-     */
-    public void pendingSplits(Map<GridHadoopInputSplit, Integer> pendingSplits) {
-        this.pendingSplits = pendingSplits;
-    }
-
-    /**
-     * Gets collection of pending splits.
-     *
-     * @return Collection of pending splits.
-     */
-    public Map<GridHadoopInputSplit, Integer> pendingSplits() {
-        return pendingSplits;
-    }
-
-    /**
-     * Sets collection of pending reducers.
-     *
-     * @param pendingReducers Collection of pending reducers.
-     */
-    public void pendingReducers(Collection<Integer> pendingReducers) {
-        this.pendingReducers = pendingReducers;
-    }
-
-    /**
-     * Gets collection of pending reducers.
-     *
-     * @return Collection of pending reducers.
-     */
-    public Collection<Integer> pendingReducers() {
-        return pendingReducers;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @param mrPlan Map-reduce plan.
-     */
-    public void mapReducePlan(GridHadoopMapReducePlan mrPlan) {
-        assert this.mrPlan == null : "Map-reduce plan can only be initialized once.";
-
-        this.mrPlan = mrPlan;
-    }
-
-    /**
-     * @return Map-reduce plan.
-     */
-    public GridHadoopMapReducePlan mapReducePlan() {
-        return mrPlan;
-    }
-
-    /**
-     * @return Job info.
-     */
-    public GridHadoopJobInfo jobInfo() {
-        return jobInfo;
-    }
-
-    /**
-     * Returns job counters.
-     *
-     * @return Collection of counters.
-     */
-    public GridHadoopCounters counters() {
-        return counters;
-    }
-
-    /**
-     * Sets counters.
-     *
-     * @param counters Collection of counters.
-     */
-    public void counters(GridHadoopCounters counters) {
-        this.counters = counters;
-    }
-
-    /**
-     * @param failCause Fail cause.
-     */
-    public void failCause(Throwable failCause) {
-        assert failCause != null;
-
-        if (this.failCause == null) // Keep the first error.
-            this.failCause = failCause;
-    }
-
-    /**
-     * @return Fail cause.
-     */
-    public Throwable failCause() {
-        return failCause;
-    }
-
-    /**
-     * @return Version.
-     */
-    public long version() {
-        return ver;
-    }
-
-    /**
-     * @param split Split.
-     * @return Task number.
-     */
-    public int taskNumber(GridHadoopInputSplit split) {
-        return pendingSplits.get(split);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeUuid(out, submitNodeId);
-        out.writeObject(jobId);
-        out.writeObject(jobInfo);
-        out.writeObject(mrPlan);
-        out.writeObject(pendingSplits);
-        out.writeObject(pendingReducers);
-        out.writeObject(phase);
-        out.writeObject(failCause);
-        out.writeLong(ver);
-        out.writeObject(reducersAddrs);
-        out.writeObject(counters);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        submitNodeId = U.readUuid(in);
-        jobId = (GridHadoopJobId)in.readObject();
-        jobInfo = (GridHadoopJobInfo)in.readObject();
-        mrPlan = (GridHadoopMapReducePlan)in.readObject();
-        pendingSplits = (Map<GridHadoopInputSplit,Integer>)in.readObject();
-        pendingReducers = (Collection<Integer>)in.readObject();
-        phase = (GridHadoopJobPhase)in.readObject();
-        failCause = (Throwable)in.readObject();
-        ver = in.readLong();
-        reducersAddrs = (Map<Integer, GridHadoopProcessDescriptor>)in.readObject();
-        counters = (GridHadoopCounters)in.readObject();
-    }
-
-    /** {@inheritDoc} */
-    public String toString() {
-        return S.toString(GridHadoopJobMetadata.class, this, "pendingMaps", pendingSplits.size(),
-            "pendingReduces", pendingReducers.size(), "failCause", failCause == null ? null :
-                failCause.getClass().getName());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
new file mode 100644
index 0000000..6042775
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
@@ -0,0 +1,305 @@
+/*
+ * 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.jobtracker;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
+
+/**
+ * Hadoop job metadata. Internal object used for distributed job state tracking.
+ */
+public class HadoopJobMetadata implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job ID. */
+    private GridHadoopJobId jobId;
+
+    /** Job info. */
+    private GridHadoopJobInfo jobInfo;
+
+    /** Node submitted job. */
+    private UUID submitNodeId;
+
+    /** Map-reduce plan. */
+    private GridHadoopMapReducePlan mrPlan;
+
+    /** Pending splits for which mapper should be executed. */
+    private Map<GridHadoopInputSplit, Integer> pendingSplits;
+
+    /** Pending reducers. */
+    private Collection<Integer> pendingReducers;
+
+    /** Reducers addresses. */
+    @GridToStringInclude
+    private Map<Integer, HadoopProcessDescriptor> reducersAddrs;
+
+    /** Job phase. */
+    private GridHadoopJobPhase phase = PHASE_SETUP;
+
+    /** Fail cause. */
+    @GridToStringExclude
+    private Throwable failCause;
+
+    /** Version. */
+    private long ver;
+
+    /** Job counters */
+    private GridHadoopCounters counters = new HadoopCountersImpl();
+
+    /**
+     * Empty constructor required by {@link Externalizable}.
+     */
+    public HadoopJobMetadata() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param submitNodeId Submit node ID.
+     * @param jobId Job ID.
+     * @param jobInfo Job info.
+     */
+    public HadoopJobMetadata(UUID submitNodeId, GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+        this.jobId = jobId;
+        this.jobInfo = jobInfo;
+        this.submitNodeId = submitNodeId;
+    }
+
+    /**
+     * Copy constructor.
+     *
+     * @param src Metadata to copy.
+     */
+    public HadoopJobMetadata(HadoopJobMetadata src) {
+        // Make sure to preserve alphabetic order.
+        counters = src.counters;
+        failCause = src.failCause;
+        jobId = src.jobId;
+        jobInfo = src.jobInfo;
+        mrPlan = src.mrPlan;
+        pendingSplits = src.pendingSplits;
+        pendingReducers = src.pendingReducers;
+        phase = src.phase;
+        reducersAddrs = src.reducersAddrs;
+        submitNodeId = src.submitNodeId;
+        ver = src.ver + 1;
+    }
+
+    /**
+     * @return Submit node ID.
+     */
+    public UUID submitNodeId() {
+        return submitNodeId;
+    }
+
+    /**
+     * @param phase Job phase.
+     */
+    public void phase(GridHadoopJobPhase phase) {
+        this.phase = phase;
+    }
+
+    /**
+     * @return Job phase.
+     */
+    public GridHadoopJobPhase phase() {
+        return phase;
+    }
+
+    /**
+     * Gets reducers addresses for external execution.
+     *
+     * @return Reducers addresses.
+     */
+    public Map<Integer, HadoopProcessDescriptor> reducersAddresses() {
+        return reducersAddrs;
+    }
+
+    /**
+     * Sets reducers addresses for external execution.
+     *
+     * @param reducersAddrs Map of addresses.
+     */
+    public void reducersAddresses(Map<Integer, HadoopProcessDescriptor> reducersAddrs) {
+        this.reducersAddrs = reducersAddrs;
+    }
+
+    /**
+     * Sets collection of pending splits.
+     *
+     * @param pendingSplits Collection of pending splits.
+     */
+    public void pendingSplits(Map<GridHadoopInputSplit, Integer> pendingSplits) {
+        this.pendingSplits = pendingSplits;
+    }
+
+    /**
+     * Gets collection of pending splits.
+     *
+     * @return Collection of pending splits.
+     */
+    public Map<GridHadoopInputSplit, Integer> pendingSplits() {
+        return pendingSplits;
+    }
+
+    /**
+     * Sets collection of pending reducers.
+     *
+     * @param pendingReducers Collection of pending reducers.
+     */
+    public void pendingReducers(Collection<Integer> pendingReducers) {
+        this.pendingReducers = pendingReducers;
+    }
+
+    /**
+     * Gets collection of pending reducers.
+     *
+     * @return Collection of pending reducers.
+     */
+    public Collection<Integer> pendingReducers() {
+        return pendingReducers;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @param mrPlan Map-reduce plan.
+     */
+    public void mapReducePlan(GridHadoopMapReducePlan mrPlan) {
+        assert this.mrPlan == null : "Map-reduce plan can only be initialized once.";
+
+        this.mrPlan = mrPlan;
+    }
+
+    /**
+     * @return Map-reduce plan.
+     */
+    public GridHadoopMapReducePlan mapReducePlan() {
+        return mrPlan;
+    }
+
+    /**
+     * @return Job info.
+     */
+    public GridHadoopJobInfo jobInfo() {
+        return jobInfo;
+    }
+
+    /**
+     * Returns job counters.
+     *
+     * @return Collection of counters.
+     */
+    public GridHadoopCounters counters() {
+        return counters;
+    }
+
+    /**
+     * Sets counters.
+     *
+     * @param counters Collection of counters.
+     */
+    public void counters(GridHadoopCounters counters) {
+        this.counters = counters;
+    }
+
+    /**
+     * @param failCause Fail cause.
+     */
+    public void failCause(Throwable failCause) {
+        assert failCause != null;
+
+        if (this.failCause == null) // Keep the first error.
+            this.failCause = failCause;
+    }
+
+    /**
+     * @return Fail cause.
+     */
+    public Throwable failCause() {
+        return failCause;
+    }
+
+    /**
+     * @return Version.
+     */
+    public long version() {
+        return ver;
+    }
+
+    /**
+     * @param split Split.
+     * @return Task number.
+     */
+    public int taskNumber(GridHadoopInputSplit split) {
+        return pendingSplits.get(split);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeUuid(out, submitNodeId);
+        out.writeObject(jobId);
+        out.writeObject(jobInfo);
+        out.writeObject(mrPlan);
+        out.writeObject(pendingSplits);
+        out.writeObject(pendingReducers);
+        out.writeObject(phase);
+        out.writeObject(failCause);
+        out.writeLong(ver);
+        out.writeObject(reducersAddrs);
+        out.writeObject(counters);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        submitNodeId = U.readUuid(in);
+        jobId = (GridHadoopJobId)in.readObject();
+        jobInfo = (GridHadoopJobInfo)in.readObject();
+        mrPlan = (GridHadoopMapReducePlan)in.readObject();
+        pendingSplits = (Map<GridHadoopInputSplit,Integer>)in.readObject();
+        pendingReducers = (Collection<Integer>)in.readObject();
+        phase = (GridHadoopJobPhase)in.readObject();
+        failCause = (Throwable)in.readObject();
+        ver = in.readLong();
+        reducersAddrs = (Map<Integer, HadoopProcessDescriptor>)in.readObject();
+        counters = (GridHadoopCounters)in.readObject();
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+        return S.toString(HadoopJobMetadata.class, this, "pendingMaps", pendingSplits.size(),
+            "pendingReduces", pendingReducers.size(), "failCause", failCause == null ? null :
+                failCause.getClass().getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 91a2d6f..a0ae3f6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -46,7 +46,7 @@ import java.util.concurrent.atomic.*;
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
 import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
-import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
+import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.*;
 
 /**
  * Hadoop job tracker.
@@ -56,10 +56,10 @@ public class HadoopJobTracker extends HadoopComponent {
     private final GridMutex mux = new GridMutex();
 
     /** */
-    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaPrj;
+    private volatile GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> jobMetaPrj;
 
     /** Projection with expiry policy for finished job updates. */
-    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaPrj;
+    private volatile GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> finishedJobMetaPrj;
 
     /** Map-reduce execution planner. */
     @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
@@ -106,8 +106,8 @@ public class HadoopJobTracker extends HadoopComponent {
      * @return Job meta projection.
      */
     @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaCache() {
-        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = jobMetaPrj;
+    private GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> jobMetaCache() {
+        GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> prj = jobMetaPrj;
 
         if (prj == null) {
             synchronized (mux) {
@@ -128,8 +128,8 @@ public class HadoopJobTracker extends HadoopComponent {
                         throw new IllegalStateException(e);
                     }
 
-                    jobMetaPrj = prj = (GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata>)
-                        sysCache.projection(GridHadoopJobId.class, GridHadoopJobMetadata.class);
+                    jobMetaPrj = prj = (GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata>)
+                        sysCache.projection(GridHadoopJobId.class, HadoopJobMetadata.class);
 
                     if (ctx.configuration().getFinishedJobInfoTtl() > 0) {
                         ExpiryPolicy finishedJobPlc = new ModifiedExpiryPolicy(
@@ -149,8 +149,8 @@ public class HadoopJobTracker extends HadoopComponent {
     /**
      * @return Projection with expiry policy for finished job updates.
      */
-    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaCache() {
-        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = finishedJobMetaPrj;
+    private GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> finishedJobMetaCache() {
+        GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> prj = finishedJobMetaPrj;
 
         if (prj == null) {
             jobMetaCache();
@@ -169,9 +169,9 @@ public class HadoopJobTracker extends HadoopComponent {
         super.onKernalStart();
 
         jobMetaCache().context().continuousQueries().executeInternalQuery(
-            new CacheEntryUpdatedListener<GridHadoopJobId, GridHadoopJobMetadata>() {
+            new CacheEntryUpdatedListener<GridHadoopJobId, HadoopJobMetadata>() {
                 @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends GridHadoopJobId,
-                    ? extends GridHadoopJobMetadata>> evts) {
+                    ? extends HadoopJobMetadata>> evts) {
                     if (!busyLock.tryReadLock())
                         return;
 
@@ -250,7 +250,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
             GridHadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
 
-            GridHadoopJobMetadata meta = new GridHadoopJobMetadata(ctx.localNodeId(), jobId, info);
+            HadoopJobMetadata meta = new HadoopJobMetadata(ctx.localNodeId(), jobId, info);
 
             meta.mapReducePlan(mrPlan);
 
@@ -268,7 +268,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
             long jobStart = U.currentTimeMillis();
 
-            GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(meta.counters(),
+            HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(meta.counters(),
                 ctx.localNodeId());
 
             perfCntr.clientSubmissionEvents(info);
@@ -297,7 +297,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @return Status.
      */
     @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public static GridHadoopJobStatus status(GridHadoopJobMetadata meta) {
+    public static GridHadoopJobStatus status(HadoopJobMetadata meta) {
         GridHadoopJobInfo jobInfo = meta.jobInfo();
 
         return new GridHadoopJobStatus(
@@ -325,7 +325,7 @@ public class HadoopJobTracker extends HadoopComponent {
             return null; // Grid is stopping.
 
         try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+            HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
             return meta != null ? status(meta) : null;
         }
@@ -346,7 +346,7 @@ public class HadoopJobTracker extends HadoopComponent {
             return null; // Grid is stopping.
 
         try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+            HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
             if (meta == null)
                 return null;
@@ -400,7 +400,7 @@ public class HadoopJobTracker extends HadoopComponent {
             return null;
 
         try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+            HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
             if (meta != null)
                 return meta.mapReducePlan();
@@ -419,7 +419,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param status Task status.
      */
     @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
-    public void onTaskFinished(GridHadoopTaskInfo info, GridHadoopTaskStatus status) {
+    public void onTaskFinished(GridHadoopTaskInfo info, HadoopTaskStatus status) {
         if (!busyLock.tryReadLock())
             return;
 
@@ -470,7 +470,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                 case COMMIT:
                 case ABORT: {
-                    GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> cache = finishedJobMetaCache();
+                    GridCacheProjectionEx<GridHadoopJobId, HadoopJobMetadata> cache = finishedJobMetaCache();
 
                     cache.invokeAsync(info.jobId(), new UpdatePhaseProcessor(incrCntrs, PHASE_COMPLETE)).
                         listenAsync(failsLog);
@@ -488,7 +488,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param jobId Job id.
      * @param c Closure of operation.
      */
-    private void transform(GridHadoopJobId jobId, EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void> c) {
+    private void transform(GridHadoopJobId jobId, EntryProcessor<GridHadoopJobId, HadoopJobMetadata, Void> c) {
         jobMetaCache().invokeAsync(jobId, c).listenAsync(failsLog);
     }
 
@@ -500,7 +500,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param desc Process descriptor.
      */
     public void onExternalMappersInitialized(GridHadoopJobId jobId, Collection<Integer> reducers,
-        GridHadoopProcessDescriptor desc) {
+        HadoopProcessDescriptor desc) {
         transform(jobId, new InitializeReducersProcessor(null, reducers, desc));
     }
 
@@ -557,7 +557,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
             // Iteration over all local entries is correct since system cache is REPLICATED.
             for (Object metaObj : jobMetaCache().values()) {
-                GridHadoopJobMetadata meta = (GridHadoopJobMetadata)metaObj;
+                HadoopJobMetadata meta = (HadoopJobMetadata)metaObj;
 
                 GridHadoopJobId jobId = meta.jobId();
 
@@ -626,13 +626,13 @@ public class HadoopJobTracker extends HadoopComponent {
      * @throws IgniteCheckedException If failed.
      */
     private void processJobMetadataUpdates(
-        Iterable<CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata>> updated)
+        Iterable<CacheEntryEvent<? extends GridHadoopJobId, ? extends HadoopJobMetadata>> updated)
         throws IgniteCheckedException {
         UUID locNodeId = ctx.localNodeId();
 
-        for (CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata> entry : updated) {
+        for (CacheEntryEvent<? extends GridHadoopJobId, ? extends HadoopJobMetadata> entry : updated) {
             GridHadoopJobId jobId = entry.getKey();
-            GridHadoopJobMetadata meta = entry.getValue();
+            HadoopJobMetadata meta = entry.getValue();
 
             if (meta == null || !ctx.isParticipating(meta))
                 continue;
@@ -689,7 +689,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param locNodeId Local node ID.
      * @throws IgniteCheckedException If failed.
      */
-    private void processJobMetaUpdate(GridHadoopJobId jobId, GridHadoopJobMetadata meta, UUID locNodeId)
+    private void processJobMetaUpdate(GridHadoopJobId jobId, HadoopJobMetadata meta, UUID locNodeId)
         throws IgniteCheckedException {
         JobLocalState state = activeJobs.get(jobId);
 
@@ -879,7 +879,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param meta Job metadata.
      * @return Collection of created task infos or {@code null} if no mapper tasks scheduled for local node.
      */
-    private Collection<GridHadoopTaskInfo> mapperTasks(Iterable<GridHadoopInputSplit> mappers, GridHadoopJobMetadata meta) {
+    private Collection<GridHadoopTaskInfo> mapperTasks(Iterable<GridHadoopInputSplit> mappers, HadoopJobMetadata meta) {
         UUID locNodeId = ctx.localNodeId();
         GridHadoopJobId jobId = meta.jobId();
 
@@ -978,7 +978,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
         try {
             if (jobInfo == null) {
-                GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+                HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
                 if (meta == null)
                     throw new IgniteCheckedException("Failed to find job metadata for ID: " + jobId);
@@ -1024,7 +1024,7 @@ public class HadoopJobTracker extends HadoopComponent {
             return false; // Grid is stopping.
 
         try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+            HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
             if (meta != null && meta.phase() != PHASE_COMPLETE && meta.phase() != PHASE_CANCELLING) {
                 HadoopTaskCancelledException err = new HadoopTaskCancelledException("Job cancelled.");
@@ -1063,7 +1063,7 @@ public class HadoopJobTracker extends HadoopComponent {
             return null;
 
         try {
-            final GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
+            final HadoopJobMetadata meta = jobMetaCache().get(jobId);
 
             return meta != null ? meta.counters() : null;
         }
@@ -1158,7 +1158,7 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param status Task status.
          * @param prev Previous closure.
          */
-        private void onSetupFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
+        private void onSetupFinished(final GridHadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
             final GridHadoopJobId jobId = taskInfo.jobId();
 
             if (status.state() == FAILED || status.state() == CRASHED)
@@ -1172,7 +1172,7 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param status Task status.
          * @param prev Previous closure.
          */
-        private void onMapFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
+        private void onMapFinished(final GridHadoopTaskInfo taskInfo, HadoopTaskStatus status,
             final StackedProcessor prev) {
             final GridHadoopJobId jobId = taskInfo.jobId();
 
@@ -1213,7 +1213,7 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param status Task status.
          * @param prev Previous closure.
          */
-        private void onReduceFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
+        private void onReduceFinished(GridHadoopTaskInfo taskInfo, HadoopTaskStatus status, StackedProcessor prev) {
             GridHadoopJobId jobId = taskInfo.jobId();
             if (status.state() == FAILED || status.state() == CRASHED)
                 // Fail the whole job.
@@ -1227,7 +1227,7 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param status Task status.
          * @param prev Previous closure.
          */
-        private void onCombineFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
+        private void onCombineFinished(GridHadoopTaskInfo taskInfo, HadoopTaskStatus status,
             final StackedProcessor prev) {
             final GridHadoopJobId jobId = taskInfo.jobId();
 
@@ -1302,7 +1302,7 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
             cp.phase(phase);
         }
     }
@@ -1343,7 +1343,7 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
             Map<GridHadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
 
             for (GridHadoopInputSplit s : splits)
@@ -1400,7 +1400,7 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
             Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
 
             rdcCp.remove(rdc);
@@ -1425,7 +1425,7 @@ public class HadoopJobTracker extends HadoopComponent {
         private final Collection<Integer> rdc;
 
         /** Process descriptor for reducers. */
-        private final GridHadoopProcessDescriptor desc;
+        private final HadoopProcessDescriptor desc;
 
         /**
          * @param prev Previous closure.
@@ -1434,7 +1434,7 @@ public class HadoopJobTracker extends HadoopComponent {
          */
         private InitializeReducersProcessor(@Nullable StackedProcessor prev,
             Collection<Integer> rdc,
-            GridHadoopProcessDescriptor desc) {
+            HadoopProcessDescriptor desc) {
             super(prev);
 
             assert !F.isEmpty(rdc);
@@ -1445,11 +1445,11 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            Map<Integer, GridHadoopProcessDescriptor> oldMap = meta.reducersAddresses();
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+            Map<Integer, HadoopProcessDescriptor> oldMap = meta.reducersAddresses();
 
-            Map<Integer, GridHadoopProcessDescriptor> rdcMap = oldMap == null ?
-                new HashMap<Integer, GridHadoopProcessDescriptor>() : new HashMap<>(oldMap);
+            Map<Integer, HadoopProcessDescriptor> rdcMap = oldMap == null ?
+                new HashMap<Integer, HadoopProcessDescriptor>() : new HashMap<>(oldMap);
 
             for (Integer r : rdc)
                 rdcMap.put(r, desc);
@@ -1511,7 +1511,7 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
             assert meta.phase() == PHASE_CANCELLING || err != null: "Invalid phase for cancel: " + meta;
 
             Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
@@ -1560,8 +1560,8 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            GridHadoopCounters cntrs = new GridHadoopCountersImpl(cp.counters());
+        @Override protected void update(HadoopJobMetadata meta, HadoopJobMetadata cp) {
+            GridHadoopCounters cntrs = new HadoopCountersImpl(cp.counters());
 
             cntrs.merge(counters);
 
@@ -1573,7 +1573,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * Abstract stacked closure.
      */
     private abstract static class StackedProcessor implements
-        EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void>, Serializable {
+        EntryProcessor<GridHadoopJobId, HadoopJobMetadata, Void>, Serializable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1588,8 +1588,8 @@ public class HadoopJobTracker extends HadoopComponent {
         }
 
         /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<GridHadoopJobId, GridHadoopJobMetadata> e, Object... args) {
-            GridHadoopJobMetadata val = apply(e.getValue());
+        @Override public Void process(MutableEntry<GridHadoopJobId, HadoopJobMetadata> e, Object... args) {
+            HadoopJobMetadata val = apply(e.getValue());
 
             if (val != null)
                 e.setValue(val);
@@ -1603,11 +1603,11 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param meta Old value.
          * @return New value.
          */
-        private GridHadoopJobMetadata apply(GridHadoopJobMetadata meta) {
+        private HadoopJobMetadata apply(HadoopJobMetadata meta) {
             if (meta == null)
                 return null;
 
-            GridHadoopJobMetadata cp = prev != null ? prev.apply(meta) : new GridHadoopJobMetadata(meta);
+            HadoopJobMetadata cp = prev != null ? prev.apply(meta) : new HadoopJobMetadata(meta);
 
             update(meta, cp);
 
@@ -1620,6 +1620,6 @@ public class HadoopJobTracker extends HadoopComponent {
          * @param meta Initial job metadata.
          * @param cp Copy.
          */
-        protected abstract void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp);
+        protected abstract void update(HadoopJobMetadata meta, HadoopJobMetadata cp);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/GridHadoopMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/GridHadoopMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/GridHadoopMessage.java
deleted file mode 100644
index 1670a8a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/GridHadoopMessage.java
+++ /dev/null
@@ -1,27 +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.message;
-
-import java.io.*;
-
-/**
- * Marker interface for all hadoop messages.
- */
-public interface GridHadoopMessage extends Externalizable {
-    // No-op.
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/HadoopMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/HadoopMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/HadoopMessage.java
new file mode 100644
index 0000000..cab6138
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/message/HadoopMessage.java
@@ -0,0 +1,27 @@
+/*
+ * 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.message;
+
+import java.io.*;
+
+/**
+ * Marker interface for all hadoop messages.
+ */
+public interface HadoopMessage extends Externalizable {
+    // No-op.
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlan.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlan.java
deleted file mode 100644
index 7988403..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlan.java
+++ /dev/null
@@ -1,107 +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.planner;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Map-reduce plan.
- */
-public class GridHadoopDefaultMapReducePlan implements GridHadoopMapReducePlan {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Mappers map. */
-    private Map<UUID, Collection<GridHadoopInputSplit>> mappers;
-
-    /** Reducers map. */
-    private Map<UUID, int[]> reducers;
-
-    /** Mappers count. */
-    private int mappersCnt;
-
-    /** Reducers count. */
-    private int reducersCnt;
-
-    /**
-     * @param mappers Mappers map.
-     * @param reducers Reducers map.
-     */
-    public GridHadoopDefaultMapReducePlan(Map<UUID, Collection<GridHadoopInputSplit>> mappers,
-        Map<UUID, int[]> reducers) {
-        this.mappers = mappers;
-        this.reducers = reducers;
-
-        if (mappers != null) {
-            for (Collection<GridHadoopInputSplit> splits : mappers.values())
-                mappersCnt += splits.size();
-        }
-
-        if (reducers != null) {
-            for (int[] rdcrs : reducers.values())
-                reducersCnt += rdcrs.length;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public int mappers() {
-        return mappersCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int reducers() {
-        return reducersCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID nodeForReducer(int reducer) {
-        assert reducer >= 0 && reducer < reducersCnt : reducer;
-
-        for (Map.Entry<UUID, int[]> entry : reducers.entrySet()) {
-            for (int r : entry.getValue()) {
-                if (r == reducer)
-                    return entry.getKey();
-            }
-        }
-
-        throw new IllegalStateException("Not found reducer index: " + reducer);
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public Collection<GridHadoopInputSplit> mappers(UUID nodeId) {
-        return mappers.get(nodeId);
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public int[] reducers(UUID nodeId) {
-        return reducers.get(nodeId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<UUID> mapperNodeIds() {
-        return mappers.keySet();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<UUID> reducerNodeIds() {
-        return reducers.keySet();
-    }
-}


[27/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
new file mode 100644
index 0000000..ab5fa68
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
@@ -0,0 +1,201 @@
+/*
+ * 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.igfs.hadoop;
+
+import org.apache.commons.logging.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.igfs.common.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * IGFS Hadoop output stream implementation.
+ */
+public class HadoopOutputIgfsStream extends OutputStream implements HadoopIgfsStreamEventListener {
+    /** Log instance. */
+    private Log log;
+
+    /** Client logger. */
+    private IgfsLogger clientLog;
+
+    /** Log stream ID. */
+    private long logStreamId;
+
+    /** Server stream delegate. */
+    private HadoopIgfsStreamDelegate delegate;
+
+    /** Closed flag. */
+    private volatile boolean closed;
+
+    /** Flag set if stream was closed due to connection breakage. */
+    private boolean connBroken;
+
+    /** Error message. */
+    private volatile String errMsg;
+
+    /** Read time. */
+    private long writeTime;
+
+    /** User time. */
+    private long userTime;
+
+    /** Last timestamp. */
+    private long lastTs;
+
+    /** Amount of written bytes. */
+    private long total;
+
+    /**
+     * Creates light output stream.
+     *
+     * @param delegate Server stream delegate.
+     * @param log Logger to use.
+     * @param clientLog Client logger.
+     */
+    public HadoopOutputIgfsStream(HadoopIgfsStreamDelegate delegate, Log log,
+        IgfsLogger clientLog, long logStreamId) {
+        this.delegate = delegate;
+        this.log = log;
+        this.clientLog = clientLog;
+        this.logStreamId = logStreamId;
+
+        lastTs = System.nanoTime();
+
+        delegate.hadoop().addEventListener(delegate, this);
+    }
+
+    /**
+     * Read start.
+     */
+    private void writeStart() {
+        long now = System.nanoTime();
+
+        userTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /**
+     * Read end.
+     */
+    private void writeEnd() {
+        long now = System.nanoTime();
+
+        writeTime += now - lastTs;
+
+        lastTs = now;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
+        check();
+
+        writeStart();
+
+        try {
+            delegate.hadoop().writeData(delegate, b, off, len);
+
+            total += len;
+        }
+        finally {
+            writeEnd();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(int b) throws IOException {
+        write(new byte[] {(byte)b});
+
+        total++;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void flush() throws IOException {
+        delegate.hadoop().flush(delegate);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        if (!closed) {
+            if (log.isDebugEnabled())
+                log.debug("Closing output stream: " + delegate);
+
+            writeStart();
+
+            delegate.hadoop().closeStream(delegate);
+
+            markClosed(false);
+
+            writeEnd();
+
+            if (clientLog.isLogEnabled())
+                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
+
+            if (log.isDebugEnabled())
+                log.debug("Closed output stream [delegate=" + delegate + ", writeTime=" + writeTime / 1000 +
+                    ", userTime=" + userTime / 1000 + ']');
+        }
+        else if(connBroken)
+            throw new IOException(
+                "Failed to close stream, because connection was broken (data could have been lost).");
+    }
+
+    /**
+     * Marks stream as closed.
+     *
+     * @param connBroken {@code True} if connection with server was lost.
+     */
+    private void markClosed(boolean connBroken) {
+        // It is ok to have race here.
+        if (!closed) {
+            closed = true;
+
+            delegate.hadoop().removeEventListener(delegate);
+
+            this.connBroken = connBroken;
+        }
+    }
+
+    /**
+     * @throws IOException If check failed.
+     */
+    private void check() throws IOException {
+        String errMsg0 = errMsg;
+
+        if (errMsg0 != null)
+            throw new IOException(errMsg0);
+
+        if (closed) {
+            if (connBroken)
+                throw new IOException("Server connection was lost.");
+            else
+                throw new IOException("Stream is closed.");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClose() throws IgniteCheckedException {
+        markClosed(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onError(String errMsg) {
+        this.errMsg = errMsg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoop.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoop.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoop.java
deleted file mode 100644
index 27d6e33..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoop.java
+++ /dev/null
@@ -1,198 +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.igfs.hadoop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Facade for communication with grid.
- */
-public interface IgfsHadoop {
-    /**
-     * Perform handshake.
-     *
-     * @param logDir Log directory.
-     * @return Future with handshake result.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException, IOException;
-
-    /**
-     * Close connection.
-     *
-     * @param force Force flag.
-     */
-    public void close(boolean force);
-
-    /**
-     * Command to retrieve file info for some IGFS path.
-     *
-     * @param path Path to get file info for.
-     * @return Future for info operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsFile info(IgfsPath path) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to update file properties.
-     *
-     * @param path IGFS path to update properties.
-     * @param props Properties to update.
-     * @return Future for update operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException, IOException;
-
-    /**
-     * Sets last access time and last modification time for a file.
-     *
-     * @param path Path to update times.
-     * @param accessTime Last access time to set.
-     * @param modificationTime Last modification time to set.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException,
-        IOException;
-
-    /**
-     * Command to rename given path.
-     *
-     * @param src Source path.
-     * @param dest Destination path.
-     * @return Future for rename operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to delete given path.
-     *
-     * @param path Path to delete.
-     * @param recursive {@code True} if deletion is recursive.
-     * @return Future for delete operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to get affinity for given path, offset and length.
-     *
-     * @param path Path to get affinity for.
-     * @param start Start position (offset).
-     * @param len Data length.
-     * @return Future for affinity command.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) throws IgniteCheckedException,
-        IOException;
-
-    /**
-     * Gets path summary.
-     *
-     * @param path Path to get summary for.
-     * @return Future that will be completed when summary is received.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to create directories.
-     *
-     * @param path Path to create.
-     * @return Future for mkdirs operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to get list of files in directory.
-     *
-     * @param path Path to list.
-     * @return Future for listFiles operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to get directory listing.
-     *
-     * @param path Path to list.
-     * @return Future for listPaths operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException, IOException;
-
-    /**
-     * Performs status request.
-     *
-     * @return Status response.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsStatus fsStatus() throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to open file for reading.
-     *
-     * @param path File path to open.
-     * @return Future for open operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsHadoopStreamDelegate open(IgfsPath path) throws IgniteCheckedException, IOException;
-
-    /**
-     * Command to open file for reading.
-     *
-     * @param path File path to open.
-     * @return Future for open operation.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsHadoopStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch) throws IgniteCheckedException,
-        IOException;
-
-    /**
-     * Command to create file and open it for output.
-     *
-     * @param path Path to file.
-     * @param overwrite If {@code true} then old file contents will be lost.
-     * @param colocate If {@code true} and called on data node, file will be written on that node.
-     * @param replication Replication factor.
-     * @param props File properties for creation.
-     * @return Stream descriptor.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsHadoopStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException, IOException;
-
-    /**
-     * Open file for output appending data to the end of a file.
-     *
-     * @param path Path to file.
-     * @param create If {@code true}, file will be created if does not exist.
-     * @param props File properties.
-     * @return Stream descriptor.
-     * @throws IgniteCheckedException If failed.
-     */
-    public IgfsHadoopStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException, IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopCommunicationException.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopCommunicationException.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopCommunicationException.java
deleted file mode 100644
index 03bf733..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopCommunicationException.java
+++ /dev/null
@@ -1,57 +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.igfs.hadoop;
-
-import org.apache.ignite.*;
-
-/**
- * Communication exception indicating a problem between file system and IGFS instance.
- */
-public class IgfsHadoopCommunicationException extends IgniteCheckedException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     * Creates new exception with given throwable as a nested cause and
-     * source of error message.
-     *
-     * @param cause Non-null throwable cause.
-     */
-    public IgfsHadoopCommunicationException(Exception cause) {
-        super(cause);
-    }
-
-    /**
-     * Creates a new exception with given error message and optional nested cause exception.
-     *
-     * @param msg Error message.
-     */
-    public IgfsHadoopCommunicationException(String msg) {
-        super(msg);
-    }
-
-    /**
-     * Creates a new exception with given error message and optional nested cause exception.
-     *
-     * @param msg Error message.
-     * @param cause Cause.
-     */
-    public IgfsHadoopCommunicationException(String msg, Exception cause) {
-        super(msg, cause);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEndpoint.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEndpoint.java
deleted file mode 100644
index 35638ea..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEndpoint.java
+++ /dev/null
@@ -1,210 +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.igfs.hadoop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
-
-/**
- * IGFS endpoint abstraction.
- */
-public class IgfsHadoopEndpoint {
-    /** Localhost. */
-    public static final String LOCALHOST = "127.0.0.1";
-
-    /** IGFS name. */
-    private final String igfsName;
-
-    /** Grid name. */
-    private final String gridName;
-
-    /** Host. */
-    private final String host;
-
-    /** Port. */
-    private final int port;
-
-    /**
-     * Normalize IGFS URI.
-     *
-     * @param uri URI.
-     * @return Normalized URI.
-     * @throws IOException If failed.
-     */
-    public static URI normalize(URI uri) throws IOException {
-        try {
-            if (!F.eq(IgniteFs.IGFS_SCHEME, uri.getScheme()))
-                throw new IOException("Failed to normalize UIR because it has non IGFS scheme: " + uri);
-
-            IgfsHadoopEndpoint endpoint = new IgfsHadoopEndpoint(uri.getAuthority());
-
-            StringBuilder sb = new StringBuilder();
-
-            if (endpoint.igfs() != null)
-                sb.append(endpoint.igfs());
-
-            if (endpoint.grid() != null)
-                sb.append(":").append(endpoint.grid());
-
-            return new URI(uri.getScheme(), sb.length() != 0 ? sb.toString() : null, endpoint.host(), endpoint.port(),
-                uri.getPath(), uri.getQuery(), uri.getFragment());
-        }
-        catch (URISyntaxException | IgniteCheckedException e) {
-            throw new IOException("Failed to normalize URI: " + uri, e);
-        }
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param connStr Connection string.
-     * @throws IgniteCheckedException If failed to parse connection string.
-     */
-    public IgfsHadoopEndpoint(@Nullable String connStr) throws IgniteCheckedException {
-        if (connStr == null)
-            connStr = "";
-
-        String[] tokens = connStr.split("@", -1);
-
-        IgniteBiTuple<String, Integer> hostPort;
-
-        if (tokens.length == 1) {
-            igfsName = null;
-            gridName = null;
-
-            hostPort = hostPort(connStr, connStr);
-        }
-        else if (tokens.length == 2) {
-            String authStr = tokens[0];
-
-            if (authStr.isEmpty()) {
-                gridName = null;
-                igfsName = null;
-            }
-            else {
-                String[] authTokens = authStr.split(":", -1);
-
-                igfsName = F.isEmpty(authTokens[0]) ? null : authTokens[0];
-
-                if (authTokens.length == 1)
-                    gridName = null;
-                else if (authTokens.length == 2)
-                    gridName = F.isEmpty(authTokens[1]) ? null : authTokens[1];
-                else
-                    throw new IgniteCheckedException("Invalid connection string format: " + connStr);
-            }
-
-            hostPort = hostPort(connStr, tokens[1]);
-        }
-        else
-            throw new IgniteCheckedException("Invalid connection string format: " + connStr);
-
-        host = hostPort.get1();
-
-        assert hostPort.get2() != null;
-
-        port = hostPort.get2();
-    }
-
-    /**
-     * Parse host and port.
-     *
-     * @param connStr Full connection string.
-     * @param hostPortStr Host/port connection string part.
-     * @return Tuple with host and port.
-     * @throws IgniteCheckedException If failed to parse connection string.
-     */
-    private IgniteBiTuple<String, Integer> hostPort(String connStr, String hostPortStr) throws IgniteCheckedException {
-        String[] tokens = hostPortStr.split(":", -1);
-
-        String host = tokens[0];
-
-        if (F.isEmpty(host))
-            host = LOCALHOST;
-
-        int port;
-
-        if (tokens.length == 1)
-            port = DFLT_IPC_PORT;
-        else if (tokens.length == 2) {
-            String portStr = tokens[1];
-
-            try {
-                port = Integer.valueOf(portStr);
-
-                if (port < 0 || port > 65535)
-                    throw new IgniteCheckedException("Invalid port number: " + connStr);
-            }
-            catch (NumberFormatException e) {
-                throw new IgniteCheckedException("Invalid port number: " + connStr);
-            }
-        }
-        else
-            throw new IgniteCheckedException("Invalid connection string format: " + connStr);
-
-        return F.t(host, port);
-    }
-
-    /**
-     * @return IGFS name.
-     */
-    @Nullable public String igfs() {
-        return igfsName;
-    }
-
-    /**
-     * @return Grid name.
-     */
-    @Nullable public String grid() {
-        return gridName;
-    }
-
-    /**
-     * @return Host.
-     */
-    public String host() {
-        return host;
-    }
-
-    /**
-     * @return Host.
-     */
-    public boolean isLocal() {
-        return F.eq(LOCALHOST, host);
-    }
-
-    /**
-     * @return Port.
-     */
-    public int port() {
-        return port;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgfsHadoopEndpoint.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEx.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEx.java
deleted file mode 100644
index da86e37..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopEx.java
+++ /dev/null
@@ -1,88 +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.igfs.hadoop;
-
-import org.apache.ignite.internal.util.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Extended IGFS server interface.
- */
-public interface IgfsHadoopEx extends IgfsHadoop {
-    /**
-     * Adds event listener that will be invoked when connection with server is lost or remote error has occurred.
-     * If connection is closed already, callback will be invoked synchronously inside this method.
-     *
-     * @param delegate Stream delegate.
-     * @param lsnr Event listener.
-     */
-    public void addEventListener(IgfsHadoopStreamDelegate delegate, IgfsHadoopStreamEventListener lsnr);
-
-    /**
-     * Removes event listener that will be invoked when connection with server is lost or remote error has occurred.
-     *
-     * @param delegate Stream delegate.
-     */
-    public void removeEventListener(IgfsHadoopStreamDelegate delegate);
-
-    /**
-     * Asynchronously reads specified amount of bytes from opened input stream.
-     *
-     * @param delegate Stream delegate.
-     * @param pos Position to read from.
-     * @param len Data length to read.
-     * @param outBuf Optional output buffer. If buffer length is less then {@code len}, all remaining
-     *     bytes will be read into new allocated buffer of length {len - outBuf.length} and this buffer will
-     *     be the result of read future.
-     * @param outOff Output offset.
-     * @param outLen Output length.
-     * @return Read data.
-     */
-    public GridPlainFuture<byte[]> readData(IgfsHadoopStreamDelegate delegate, long pos, int len,
-        @Nullable final byte[] outBuf, final int outOff, final int outLen);
-
-    /**
-     * Writes data to the stream with given streamId. This method does not return any future since
-     * no response to write request is sent.
-     *
-     * @param delegate Stream delegate.
-     * @param data Data to write.
-     * @param off Offset.
-     * @param len Length.
-     * @throws IOException If failed.
-     */
-    public void writeData(IgfsHadoopStreamDelegate delegate, byte[] data, int off, int len) throws IOException;
-
-    /**
-     * Close server stream.
-     *
-     * @param delegate Stream delegate.
-     * @throws IOException If failed.
-     */
-    public void closeStream(IgfsHadoopStreamDelegate delegate) throws IOException;
-
-    /**
-     * Flush output stream.
-     *
-     * @param delegate Stream delegate.
-     * @throws IOException If failed.
-     */
-    public void flush(IgfsHadoopStreamDelegate delegate) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFSProperties.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFSProperties.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFSProperties.java
deleted file mode 100644
index c9d1322..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFSProperties.java
+++ /dev/null
@@ -1,88 +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.igfs.hadoop;
-
-import org.apache.hadoop.fs.permission.*;
-import org.apache.ignite.*;
-
-import java.util.*;
-
-import static org.apache.ignite.IgniteFs.*;
-
-/**
- * Hadoop file system properties.
- */
-public class IgfsHadoopFSProperties {
-    /** Username. */
-    private String usrName;
-
-    /** Group name. */
-    private String grpName;
-
-    /** Permissions. */
-    private FsPermission perm;
-
-    /**
-     * Constructor.
-     *
-     * @param props Properties.
-     * @throws IgniteException In case of error.
-     */
-    public IgfsHadoopFSProperties(Map<String, String> props) throws IgniteException {
-        usrName = props.get(PROP_USER_NAME);
-        grpName = props.get(PROP_GROUP_NAME);
-
-        String permStr = props.get(PROP_PERMISSION);
-
-        if (permStr != null) {
-            try {
-                perm = new FsPermission((short)Integer.parseInt(permStr, 8));
-            }
-            catch (NumberFormatException ignore) {
-                throw new IgniteException("Permissions cannot be parsed: " + permStr);
-            }
-        }
-    }
-
-    /**
-     * Get user name.
-     *
-     * @return User name.
-     */
-    public String userName() {
-        return usrName;
-    }
-
-    /**
-     * Get group name.
-     *
-     * @return Group name.
-     */
-    public String groupName() {
-        return grpName;
-    }
-
-    /**
-     * Get permission.
-     *
-     * @return Permission.
-     */
-    public FsPermission permission() {
-        return perm;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFuture.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFuture.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFuture.java
deleted file mode 100644
index 476641c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopFuture.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.igfs.hadoop;
-
-import org.apache.ignite.internal.util.lang.*;
-import org.jetbrains.annotations.*;
-
-/**
- * IGFS client future that holds response parse closure.
- */
-public class IgfsHadoopFuture<T> extends GridPlainFutureAdapter<T> {
-    /** Output buffer. */
-    private byte[] outBuf;
-
-    /** Output offset. */
-    private int outOff;
-
-    /** Output length. */
-    private int outLen;
-
-    /** Read future flag. */
-    private boolean read;
-
-    /**
-     * @return Output buffer.
-     */
-    public byte[] outputBuffer() {
-        return outBuf;
-    }
-
-    /**
-     * @param outBuf Output buffer.
-     */
-    public void outputBuffer(@Nullable byte[] outBuf) {
-        this.outBuf = outBuf;
-    }
-
-    /**
-     * @return Offset in output buffer to write from.
-     */
-    public int outputOffset() {
-        return outOff;
-    }
-
-    /**
-     * @param outOff Offset in output buffer to write from.
-     */
-    public void outputOffset(int outOff) {
-        this.outOff = outOff;
-    }
-
-    /**
-     * @return Length to write to output buffer.
-     */
-    public int outputLength() {
-        return outLen;
-    }
-
-    /**
-     * @param outLen Length to write to output buffer.
-     */
-    public void outputLength(int outLen) {
-        this.outLen = outLen;
-    }
-
-    /**
-     * @param read {@code True} if this is a read future.
-     */
-    public void read(boolean read) {
-        this.read = read;
-    }
-
-    /**
-     * @return {@code True} if this is a read future.
-     */
-    public boolean read() {
-        return read;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInProc.java
deleted file mode 100644
index 8245125..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInProc.java
+++ /dev/null
@@ -1,409 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Communication with grid in the same process.
- */
-public class IgfsHadoopInProc implements IgfsHadoopEx {
-    /** Target IGFS. */
-    private final IgfsEx igfs;
-
-    /** Buffer size. */
-    private final int bufSize;
-
-    /** Event listeners. */
-    private final Map<IgfsHadoopStreamDelegate, IgfsHadoopStreamEventListener> lsnrs =
-        new ConcurrentHashMap<>();
-
-    /** Logger. */
-    private final Log log;
-
-    /**
-     * Constructor.
-     *
-     * @param igfs Target IGFS.
-     * @param log Log.
-     */
-    public IgfsHadoopInProc(IgfsEx igfs, Log log) {
-        this.igfs = igfs;
-        this.log = log;
-
-        bufSize = igfs.configuration().getBlockSize() * 2;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) {
-        igfs.clientLogDirectory(logDir);
-
-        return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
-            igfs.globalSampling());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close(boolean force) {
-        // Perform cleanup.
-        for (IgfsHadoopStreamEventListener lsnr : lsnrs.values()) {
-            try {
-                lsnr.onClose();
-            }
-            catch (IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to notify stream event listener", e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
-        try {
-            return igfs.info(path);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to get file info because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        try {
-            return igfs.update(path, props);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to update file because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
-        try {
-            igfs.setTimes(path, accessTime, modificationTime);
-
-            return true;
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to set path times because Grid is stopping: " +
-                path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
-        try {
-            igfs.rename(src, dest);
-
-            return true;
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to rename path because Grid is stopping: " + src);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
-        try {
-            return igfs.delete(path, recursive);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to delete path because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
-        try {
-            return igfs.globalSpace();
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to get file system status because Grid is " +
-                "stopping.");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
-        try {
-            return igfs.listPaths(path);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to list paths because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
-        try {
-            return igfs.listFiles(path);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to list files because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        try {
-            igfs.mkdirs(path, props);
-
-            return true;
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to create directory because Grid is stopping: " +
-                path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
-        try {
-            return igfs.summary(path);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to get content summary because Grid is stopping: " +
-                path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
-        throws IgniteCheckedException {
-        try {
-            return igfs.affinity(path, start, len);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to get affinity because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
-        try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
-
-            return new IgfsHadoopStreamDelegate(this, stream, stream.fileInfo().length());
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to open file because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch)
-        throws IgniteCheckedException {
-        try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
-
-            return new IgfsHadoopStreamDelegate(this, stream, stream.fileInfo().length());
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to open file because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
-        try {
-            IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
-                colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
-
-            return new IgfsHadoopStreamDelegate(this, stream);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to create file because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
-        try {
-            IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
-
-            return new IgfsHadoopStreamDelegate(this, stream);
-        }
-        catch (IgniteException e) {
-            throw new IgniteCheckedException(e);
-        }
-        catch (IllegalStateException e) {
-            throw new IgfsHadoopCommunicationException("Failed to append file because Grid is stopping: " + path);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridPlainFuture<byte[]> readData(IgfsHadoopStreamDelegate delegate, long pos, int len,
-        @Nullable byte[] outBuf, int outOff, int outLen) {
-        IgfsInputStreamAdapter stream = delegate.target();
-
-        try {
-            byte[] res = null;
-
-            if (outBuf != null) {
-                int outTailLen = outBuf.length - outOff;
-
-                if (len <= outTailLen)
-                    stream.readFully(pos, outBuf, outOff, len);
-                else {
-                    stream.readFully(pos, outBuf, outOff, outTailLen);
-
-                    int remainderLen = len - outTailLen;
-
-                    res = new byte[remainderLen];
-
-                    stream.readFully(pos, res, 0, remainderLen);
-                }
-            } else {
-                res = new byte[len];
-
-                stream.readFully(pos, res, 0, len);
-            }
-
-            return new GridPlainFutureAdapter<>(res);
-        }
-        catch (IllegalStateException | IOException e) {
-            IgfsHadoopStreamEventListener lsnr = lsnrs.get(delegate);
-
-            if (lsnr != null)
-                lsnr.onError(e.getMessage());
-
-            return new GridPlainFutureAdapter<>(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeData(IgfsHadoopStreamDelegate delegate, byte[] data, int off, int len)
-        throws IOException {
-        try {
-            IgfsOutputStream stream = delegate.target();
-
-            stream.write(data, off, len);
-        }
-        catch (IllegalStateException | IOException e) {
-            IgfsHadoopStreamEventListener lsnr = lsnrs.get(delegate);
-
-            if (lsnr != null)
-                lsnr.onError(e.getMessage());
-
-            if (e instanceof IllegalStateException)
-                throw new IOException("Failed to write data to IGFS stream because Grid is stopping.", e);
-            else
-                throw e;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush(IgfsHadoopStreamDelegate delegate) throws IOException {
-        try {
-            IgfsOutputStream stream = delegate.target();
-
-            stream.flush();
-        }
-        catch (IllegalStateException | IOException e) {
-            IgfsHadoopStreamEventListener lsnr = lsnrs.get(delegate);
-
-            if (lsnr != null)
-                lsnr.onError(e.getMessage());
-
-            if (e instanceof IllegalStateException)
-                throw new IOException("Failed to flush data to IGFS stream because Grid is stopping.", e);
-            else
-                throw e;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void closeStream(IgfsHadoopStreamDelegate desc) throws IOException {
-        Closeable closeable = desc.target();
-
-        try {
-            closeable.close();
-        }
-        catch (IllegalStateException e) {
-            throw new IOException("Failed to close IGFS stream because Grid is stopping.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addEventListener(IgfsHadoopStreamDelegate delegate,
-        IgfsHadoopStreamEventListener lsnr) {
-        IgfsHadoopStreamEventListener lsnr0 = lsnrs.put(delegate, lsnr);
-
-        assert lsnr0 == null || lsnr0 == lsnr;
-
-        if (log.isDebugEnabled())
-            log.debug("Added stream event listener [delegate=" + delegate + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeEventListener(IgfsHadoopStreamDelegate delegate) {
-        IgfsHadoopStreamEventListener lsnr0 = lsnrs.remove(delegate);
-
-        if (lsnr0 != null && log.isDebugEnabled())
-            log.debug("Removed stream event listener [delegate=" + delegate + ']');
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInputStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInputStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInputStream.java
deleted file mode 100644
index efc5264..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopInputStream.java
+++ /dev/null
@@ -1,626 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * IGFS input stream wrapper for hadoop interfaces.
- */
-@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-public final class IgfsHadoopInputStream extends InputStream implements Seekable, PositionedReadable,
-    IgfsHadoopStreamEventListener {
-    /** Minimum buffer size. */
-    private static final int MIN_BUF_SIZE = 4 * 1024;
-
-    /** Server stream delegate. */
-    private IgfsHadoopStreamDelegate delegate;
-
-    /** Stream ID used by logger. */
-    private long logStreamId;
-
-    /** Stream position. */
-    private long pos;
-
-    /** Stream read limit. */
-    private long limit;
-
-    /** Mark position. */
-    private long markPos = -1;
-
-    /** Prefetch buffer. */
-    private DoubleFetchBuffer buf = new DoubleFetchBuffer();
-
-    /** Buffer half size for double-buffering. */
-    private int bufHalfSize;
-
-    /** Closed flag. */
-    private volatile boolean closed;
-
-    /** Flag set if stream was closed due to connection breakage. */
-    private boolean connBroken;
-
-    /** Logger. */
-    private Log log;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Read time. */
-    private long readTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of read bytes. */
-    private long total;
-
-    /**
-     * Creates input stream.
-     *
-     * @param delegate Server stream delegate.
-     * @param limit Read limit.
-     * @param bufSize Buffer size.
-     * @param log Log.
-     * @param clientLog Client logger.
-     */
-    public IgfsHadoopInputStream(IgfsHadoopStreamDelegate delegate, long limit, int bufSize, Log log,
-        IgfsLogger clientLog, long logStreamId) {
-        assert limit >= 0;
-
-        this.delegate = delegate;
-        this.limit = limit;
-        this.log = log;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        bufHalfSize = Math.max(bufSize, MIN_BUF_SIZE);
-
-        lastTs = System.nanoTime();
-
-        delegate.hadoop().addEventListener(delegate, this);
-    }
-
-    /**
-     * Read start.
-     */
-    private void readStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void readEnd() {
-        long now = System.nanoTime();
-
-        readTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read() throws IOException {
-        checkClosed();
-
-        readStart();
-
-        try {
-            if (eof())
-                return -1;
-
-            buf.refreshAhead(pos);
-
-            int res = buf.atPosition(pos);
-
-            pos++;
-            total++;
-
-            buf.refreshAhead(pos);
-
-            return res;
-        }
-        catch (IgniteCheckedException e) {
-            throw IgfsHadoopUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(@NotNull byte[] b, int off, int len) throws IOException {
-        checkClosed();
-
-        if (eof())
-            return -1;
-
-        readStart();
-
-        try {
-            long remaining = limit - pos;
-
-            int read = buf.flatten(b, pos, off, len);
-
-            pos += read;
-            total += read;
-            remaining -= read;
-
-            if (remaining > 0 && read != len) {
-                int readAmt = (int)Math.min(remaining, len - read);
-
-                delegate.hadoop().readData(delegate, pos, readAmt, b, off + read, len - read).get();
-
-                read += readAmt;
-                pos += readAmt;
-                total += readAmt;
-            }
-
-            buf.refreshAhead(pos);
-
-            return read;
-        }
-        catch (IgniteCheckedException e) {
-            throw IgfsHadoopUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long skip(long n) throws IOException {
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSkip(logStreamId, n);
-
-        long oldPos = pos;
-
-        if (pos + n <= limit)
-            pos += n;
-        else
-            pos = limit;
-
-        buf.refreshAhead(pos);
-
-        return pos - oldPos;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int available() throws IOException {
-        checkClosed();
-
-        int available = buf.available(pos);
-
-        assert available >= 0;
-
-        return available;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void close() throws IOException {
-        if (!closed) {
-            readStart();
-
-            if (log.isDebugEnabled())
-                log.debug("Closing input stream: " + delegate);
-
-            delegate.hadoop().closeStream(delegate);
-
-            readEnd();
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseIn(logStreamId, userTime, readTime, total);
-
-            markClosed(false);
-
-            if (log.isDebugEnabled())
-                log.debug("Closed stream [delegate=" + delegate + ", readTime=" + readTime +
-                    ", userTime=" + userTime + ']');
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void mark(int readLimit) {
-        markPos = pos;
-
-        if (clientLog.isLogEnabled())
-            clientLog.logMark(logStreamId, readLimit);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void reset() throws IOException {
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logReset(logStreamId);
-
-        if (markPos == -1)
-            throw new IOException("Stream was not marked.");
-
-        pos = markPos;
-
-        buf.refreshAhead(pos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean markSupported() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized int read(long position, byte[] buf, int off, int len) throws IOException {
-        long remaining = limit - position;
-
-        int read = (int)Math.min(len, remaining);
-
-        // Return -1 at EOF.
-        if (read == 0)
-            return -1;
-
-        readFully(position, buf, off, read);
-
-        return read;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void readFully(long position, byte[] buf, int off, int len) throws IOException {
-        long remaining = limit - position;
-
-        checkClosed();
-
-        if (len > remaining)
-            throw new EOFException("End of stream reached before data was fully read.");
-
-        readStart();
-
-        try {
-            int read = this.buf.flatten(buf, position, off, len);
-
-            total += read;
-
-            if (read != len) {
-                int readAmt = len - read;
-
-                delegate.hadoop().readData(delegate, position + read, readAmt, buf, off + read, readAmt).get();
-
-                total += readAmt;
-            }
-
-            if (clientLog.isLogEnabled())
-                clientLog.logRandomRead(logStreamId, position, len);
-        }
-        catch (IgniteCheckedException e) {
-            throw IgfsHadoopUtils.cast(e);
-        }
-        finally {
-            readEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readFully(long position, byte[] buf) throws IOException {
-        readFully(position, buf, 0, buf.length);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized void seek(long pos) throws IOException {
-        A.ensure(pos >= 0, "position must be non-negative");
-
-        checkClosed();
-
-        if (clientLog.isLogEnabled())
-            clientLog.logSeek(logStreamId, pos);
-
-        if (pos > limit)
-            pos = limit;
-
-        if (log.isDebugEnabled())
-            log.debug("Seek to position [delegate=" + delegate + ", pos=" + pos + ", oldPos=" + this.pos + ']');
-
-        this.pos = pos;
-
-        buf.refreshAhead(pos);
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized long getPos() {
-        return pos;
-    }
-
-    /** {@inheritDoc} */
-    @Override public synchronized boolean seekToNewSource(long targetPos) {
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() {
-        markClosed(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(String errMsg) {
-        // No-op.
-    }
-
-    /**
-     * Marks stream as closed.
-     *
-     * @param connBroken {@code True} if connection with server was lost.
-     */
-    private void markClosed(boolean connBroken) {
-        // It is ok to have race here.
-        if (!closed) {
-            closed = true;
-
-            this.connBroken = connBroken;
-
-            delegate.hadoop().removeEventListener(delegate);
-        }
-    }
-
-    /**
-     * @throws IOException If check failed.
-     */
-    private void checkClosed() throws IOException {
-        if (closed) {
-            if (connBroken)
-                throw new IOException("Server connection was lost.");
-            else
-                throw new IOException("Stream is closed.");
-        }
-    }
-
-    /**
-     * @return {@code True} if end of stream reached.
-     */
-    private boolean eof() {
-        return limit == pos;
-    }
-
-    /**
-     * Asynchronous prefetch buffer.
-     */
-    private static class FetchBufferPart {
-        /** Read future. */
-        private GridPlainFuture<byte[]> readFut;
-
-        /** Position of cached chunk in file. */
-        private long pos;
-
-        /** Prefetch length. Need to store as read future result might be not available yet. */
-        private int len;
-
-        /**
-         * Creates fetch buffer part.
-         *
-         * @param readFut Read future for this buffer.
-         * @param pos Read position.
-         * @param len Chunk length.
-         */
-        private FetchBufferPart(GridPlainFuture<byte[]> readFut, long pos, int len) {
-            this.readFut = readFut;
-            this.pos = pos;
-            this.len = len;
-        }
-
-        /**
-         * Copies cached data if specified position matches cached region.
-         *
-         * @param dst Destination buffer.
-         * @param pos Read position in file.
-         * @param dstOff Offset in destination buffer from which start writing.
-         * @param len Maximum number of bytes to copy.
-         * @return Number of bytes copied.
-         * @throws IgniteCheckedException If read future failed.
-         */
-        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
-            // If read start position is within cached boundaries.
-            if (contains(pos)) {
-                byte[] data = readFut.get();
-
-                int srcPos = (int)(pos - this.pos);
-                int cpLen = Math.min(len, data.length - srcPos);
-
-                U.arrayCopy(data, srcPos, dst, dstOff, cpLen);
-
-                return cpLen;
-            }
-
-            return 0;
-        }
-
-        /**
-         * @return {@code True} if data is ready to be read.
-         */
-        public boolean ready() {
-            return readFut.isDone();
-        }
-
-        /**
-         * Checks if current buffer part contains given position.
-         *
-         * @param pos Position to check.
-         * @return {@code True} if position matches buffer region.
-         */
-        public boolean contains(long pos) {
-            return this.pos <= pos && this.pos + len > pos;
-        }
-    }
-
-    private class DoubleFetchBuffer {
-        /**  */
-        private FetchBufferPart first;
-
-        /** */
-        private FetchBufferPart second;
-
-        /**
-         * Copies fetched data from both buffers to destination array if cached region matched read position.
-         *
-         * @param dst Destination buffer.
-         * @param pos Read position in file.
-         * @param dstOff Destination buffer offset.
-         * @param len Maximum number of bytes to copy.
-         * @return Number of bytes copied.
-         * @throws IgniteCheckedException If any read operation failed.
-         */
-        public int flatten(byte[] dst, long pos, int dstOff, int len) throws IgniteCheckedException {
-            assert dstOff >= 0;
-            assert dstOff + len <= dst.length : "Invalid indices [dst.length=" + dst.length + ", dstOff=" + dstOff +
-                ", len=" + len + ']';
-
-            int bytesCopied = 0;
-
-            if (first != null) {
-                bytesCopied += first.flatten(dst, pos, dstOff, len);
-
-                if (bytesCopied != len && second != null) {
-                    assert second.pos == first.pos + first.len;
-
-                    bytesCopied += second.flatten(dst, pos + bytesCopied, dstOff + bytesCopied, len - bytesCopied);
-                }
-            }
-
-            return bytesCopied;
-        }
-
-        /**
-         * Gets byte at specified position in buffer.
-         *
-         * @param pos Stream position.
-         * @return Read byte.
-         * @throws IgniteCheckedException If read failed.
-         */
-        public int atPosition(long pos) throws IgniteCheckedException {
-            // Should not reach here if stream contains no data.
-            assert first != null;
-
-            if (first.contains(pos)) {
-                byte[] bytes = first.readFut.get();
-
-                return bytes[((int)(pos - first.pos))] & 0xFF;
-            }
-            else {
-                assert second != null;
-                assert second.contains(pos);
-
-                byte[] bytes = second.readFut.get();
-
-                return bytes[((int)(pos - second.pos))] & 0xFF;
-            }
-        }
-
-        /**
-         * Starts asynchronous buffer refresh if needed, depending on current position.
-         *
-         * @param pos Current stream position.
-         */
-        public void refreshAhead(long pos) {
-            if (fullPrefetch(pos)) {
-                first = fetch(pos, bufHalfSize);
-                second = fetch(pos + bufHalfSize, bufHalfSize);
-            }
-            else if (needFlip(pos)) {
-                first = second;
-
-                second = fetch(first.pos + first.len, bufHalfSize);
-            }
-        }
-
-        /**
-         * @param pos Position from which read is expected.
-         * @return Number of bytes available to be read without blocking.
-         */
-        public int available(long pos) {
-            int available = 0;
-
-            if (first != null) {
-                if (first.contains(pos)) {
-                    if (first.ready()) {
-                        available += (pos - first.pos);
-
-                        if (second != null && second.ready())
-                            available += second.len;
-                    }
-                }
-                else {
-                    if (second != null && second.contains(pos) && second.ready())
-                        available += (pos - second.pos);
-                }
-            }
-
-            return available;
-        }
-
-        /**
-         * Checks if position shifted enough to forget previous buffer.
-         *
-         * @param pos Current position.
-         * @return {@code True} if need flip buffers.
-         */
-        private boolean needFlip(long pos) {
-            // Return true if we read more then half of second buffer.
-            return second != null && second.contains(pos);
-        }
-
-        /**
-         * Determines if all cached bytes should be discarded and new region should be
-         * prefetched.
-         *
-         * @param curPos Current stream position.
-         * @return {@code True} if need to refresh both blocks.
-         */
-        private boolean fullPrefetch(long curPos) {
-            // If no data was prefetched yet, return true.
-            return first == null || curPos < first.pos || (second != null && curPos >= second.pos + second.len);
-        }
-
-        /**
-         * Starts asynchronous fetch for given region.
-         *
-         * @param pos Position to read from.
-         * @param size Number of bytes to read.
-         * @return Fetch buffer part.
-         */
-        private FetchBufferPart fetch(long pos, int size) {
-            long remaining = limit - pos;
-
-            size = (int)Math.min(size, remaining);
-
-            return size <= 0 ? null :
-                new FetchBufferPart(delegate.hadoop().readData(delegate, pos, size, null, 0, 0), pos, size);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIo.java
deleted file mode 100644
index 46f5a6c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopIo.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.igfs.hadoop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.jetbrains.annotations.*;
-
-/**
- * IO abstraction layer for IGFS client. Two kind of messages are expected to be sent: requests with response
- * and request without response.
- */
-public interface IgfsHadoopIo {
-    /**
-     * Sends given IGFS client message and asynchronously awaits for response.
-     *
-     * @param msg Message to send.
-     * @return Future that will be completed.
-     * @throws IgniteCheckedException If a message cannot be sent (connection is broken or client was closed).
-     */
-    public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException;
-
-    /**
-     * Sends given IGFS client message and asynchronously awaits for response. When IO detects response
-     * beginning for given message it stops reading data and passes input stream to closure which can read
-     * response in a specific way.
-     *
-     * @param msg Message to send.
-     * @param outBuf Output buffer. If {@code null}, the output buffer is not used.
-     * @param outOff Output buffer offset.
-     * @param outLen Output buffer length.
-     * @return Future that will be completed when response is returned from closure.
-     * @throws IgniteCheckedException If a message cannot be sent (connection is broken or client was closed).
-     */
-    public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff, int outLen)
-        throws IgniteCheckedException;
-
-    /**
-     * Sends given message and does not wait for response.
-     *
-     * @param msg Message to send.
-     * @throws IgniteCheckedException If send failed.
-     */
-    public void sendPlain(IgfsMessage msg) throws IgniteCheckedException;
-
-    /**
-     * Adds event listener that will be invoked when connection with server is lost or remote error has occurred.
-     * If connection is closed already, callback will be invoked synchronously inside this method.
-     *
-     * @param lsnr Event listener.
-     */
-    public void addEventListener(IgfsHadoopIpcIoListener lsnr);
-
-    /**
-     * Removes event listener that will be invoked when connection with server is lost or remote error has occurred.
-     *
-     * @param lsnr Event listener.
-     */
-    public void removeEventListener(IgfsHadoopIpcIoListener lsnr);
-}


[07/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
new file mode 100644
index 0000000..2e2b5cb
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultMapReducePlannerSelfTest.java
@@ -0,0 +1,1005 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.mapreduce.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.processors.hadoop.planner.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
+
+import java.net.*;
+import java.util.*;
+
+/**
+ *
+ */
+public class HadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstractSelfTest {
+    /** */
+    private static final UUID ID_1 = new UUID(0, 1);
+
+    /** */
+    private static final UUID ID_2 = new UUID(0, 2);
+
+    /** */
+    private static final UUID ID_3 = new UUID(0, 3);
+
+    /** */
+    private static final String HOST_1 = "host1";
+
+    /** */
+    private static final String HOST_2 = "host2";
+
+    /** */
+    private static final String HOST_3 = "host3";
+
+    /** */
+    private static final String INVALID_HOST_1 = "invalid_host1";
+
+    /** */
+    private static final String INVALID_HOST_2 = "invalid_host2";
+
+    /** */
+    private static final String INVALID_HOST_3 = "invalid_host3";
+
+    /** Mocked Grid. */
+    private static final MockIgnite GRID = new MockIgnite();
+
+    /** Mocked IGFS. */
+    private static final IgniteFs IGFS = new MockIgfs();
+
+    /** Planner. */
+    private static final GridHadoopMapReducePlanner PLANNER = new HadoopDefaultMapReducePlanner();
+
+    /** Block locations. */
+    private static final Map<Block, Collection<IgfsBlockLocation>> BLOCK_MAP = new HashMap<>();
+
+    /** Proxy map. */
+    private static final Map<URI, Boolean> PROXY_MAP = new HashMap<>();
+
+    /** Last created plan. */
+    private static final ThreadLocal<GridHadoopMapReducePlan> PLAN = new ThreadLocal<>();
+
+    /**
+     *
+     */
+    static {
+        GridTestUtils.setFieldValue(PLANNER, "ignite", GRID);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        GridTestUtils.setFieldValue(PLANNER, "log", log());
+
+        BLOCK_MAP.clear();
+        PROXY_MAP.clear();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testIgfsOneBlockPerNode() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1);
+        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_2);
+        GridHadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_3);
+
+        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1));
+        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_2));
+        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_3));
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureEmpty(ID_2);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1);
+        assert ensureMappers(ID_1, split1);
+        assert ensureReducers(ID_1, 2);
+        assert ensureEmpty(ID_2);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2, split3);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureMappers(ID_3, split3);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureReducers(ID_3, 1);
+
+        plan(5, split1, split2, split3);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureMappers(ID_3, split3);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testNonIgfsOneBlockPerNode() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1);
+        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_2);
+        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_3);
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureEmpty(ID_2);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1);
+        assert ensureMappers(ID_1, split1);
+        assert ensureReducers(ID_1, 2);
+        assert ensureEmpty(ID_2);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2, split3);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureMappers(ID_3, split3);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureReducers(ID_3, 1);
+
+        plan(5, split1, split2, split3);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureMappers(ID_3, split3);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2);
+        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2);
+        GridHadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_1, HOST_3);
+
+        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1, ID_2));
+        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_1, ID_2));
+        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_1, ID_3));
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split1, split2);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1, split2);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2, split3);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureReducers(ID_3, 1);
+
+        plan(5, split1, split2, split3);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testNonIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1, HOST_2);
+        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_1, HOST_2);
+        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_1, HOST_3);
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split1, split2);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1, split2);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+
+        plan(3, split1, split2, split3);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureReducers(ID_3, 1);
+
+        plan(5, split1, split2, split3);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testIgfsSeveralComplexBlocksPerNode() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2, HOST_3);
+        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2, HOST_3);
+
+        mapIgfsBlock(split1.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_1, ID_3));
+        mapIgfsBlock(split2.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_2, ID_3));
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureEmpty(ID_2);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split2);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_1);
+        assert ensureEmpty(ID_3);
+
+        plan(1, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) || ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0);
+        assert ensureEmpty(ID_3);
+
+        plan(2, split1, split2);
+        assert ensureMappers(ID_1, split1);
+        assert ensureMappers(ID_2, split2);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureEmpty(ID_3);
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void testNonIgfsOrphans() throws IgniteCheckedException {
+        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, INVALID_HOST_1, INVALID_HOST_2);
+        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, INVALID_HOST_1, INVALID_HOST_3);
+        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, INVALID_HOST_2, INVALID_HOST_3);
+
+        plan(1, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1) && ensureEmpty(ID_3) ||
+            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 1);
+
+        plan(2, split1);
+        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
+            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2) && ensureEmpty(ID_3) ||
+            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 2);
+
+        plan(1, split1, split2, split3);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 0) ||
+            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 0) ||
+            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 1);
+
+        plan(3, split1, split2, split3);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
+        assert ensureReducers(ID_1, 1);
+        assert ensureReducers(ID_2, 1);
+        assert ensureReducers(ID_3, 1);
+
+        plan(5, split1, split2, split3);
+        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
+            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
+            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
+        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
+            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
+    }
+
+    /**
+     * Create plan.
+     *
+     * @param reducers Reducers count.
+     * @param splits Splits.
+     * @return Plan.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static GridHadoopMapReducePlan plan(int reducers, GridHadoopInputSplit... splits) throws IgniteCheckedException {
+        assert reducers > 0;
+        assert splits != null && splits.length > 0;
+
+        Collection<GridHadoopInputSplit> splitList = new ArrayList<>(splits.length);
+
+        Collections.addAll(splitList, splits);
+
+        Collection<ClusterNode> top = new ArrayList<>();
+
+        GridTestNode node1 = new GridTestNode(ID_1);
+        GridTestNode node2 = new GridTestNode(ID_2);
+        GridTestNode node3 = new GridTestNode(ID_3);
+
+        node1.setHostName(HOST_1);
+        node2.setHostName(HOST_2);
+        node3.setHostName(HOST_3);
+
+        top.add(node1);
+        top.add(node2);
+        top.add(node3);
+
+        GridHadoopMapReducePlan plan = PLANNER.preparePlan(new MockJob(reducers, splitList), top, null);
+
+        PLAN.set(plan);
+
+        return plan;
+    }
+
+    /**
+     * Ensure that node contains the given mappers.
+     *
+     * @param nodeId Node ID.
+     * @param expSplits Expected splits.
+     * @return {@code True} if this assumption is valid.
+     */
+    private static boolean ensureMappers(UUID nodeId, GridHadoopInputSplit... expSplits) {
+        Collection<GridHadoopInputSplit> expSplitsCol = new ArrayList<>();
+
+        Collections.addAll(expSplitsCol, expSplits);
+
+        Collection<GridHadoopInputSplit> splits = PLAN.get().mappers(nodeId);
+
+        return F.eq(expSplitsCol, splits);
+    }
+
+    /**
+     * Ensure that node contains the given amount of reducers.
+     *
+     * @param nodeId Node ID.
+     * @param reducers Reducers.
+     * @return {@code True} if this assumption is valid.
+     */
+    private static boolean ensureReducers(UUID nodeId, int reducers) {
+        int[] reducersArr = PLAN.get().reducers(nodeId);
+
+        return reducers == 0 ? F.isEmpty(reducersArr) : (reducersArr != null && reducersArr.length == reducers);
+    }
+
+    /**
+     * Ensure that no mappers and reducers is located on this node.
+     *
+     * @param nodeId Node ID.
+     * @return {@code True} if this assumption is valid.
+     */
+    private static boolean ensureEmpty(UUID nodeId) {
+        return F.isEmpty(PLAN.get().mappers(nodeId)) && F.isEmpty(PLAN.get().reducers(nodeId));
+    }
+
+    /**
+     * Create split.
+     *
+     * @param igfs IGFS flag.
+     * @param file File.
+     * @param start Start.
+     * @param len Length.
+     * @param hosts Hosts.
+     * @return Split.
+     */
+    private static GridHadoopFileBlock split(boolean igfs, String file, long start, long len, String... hosts) {
+        URI uri = URI.create((igfs ? "igfs://igfs@" : "hdfs://") + file);
+
+        return new GridHadoopFileBlock(hosts, uri, start, len);
+    }
+
+    /**
+     * Create block location.
+     *
+     * @param start Start.
+     * @param len Length.
+     * @param nodeIds Node IDs.
+     * @return Block location.
+     */
+    private static IgfsBlockLocation location(long start, long len, UUID... nodeIds) {
+        assert nodeIds != null && nodeIds.length > 0;
+
+        Collection<ClusterNode> nodes = new ArrayList<>(nodeIds.length);
+
+        for (UUID id : nodeIds)
+            nodes.add(new GridTestNode(id));
+
+        return new IgfsBlockLocationImpl(start, len, nodes);
+    }
+
+    /**
+     * Map IGFS block to nodes.
+     *
+     * @param file File.
+     * @param start Start.
+     * @param len Length.
+     * @param locations Locations.
+     */
+    private static void mapIgfsBlock(URI file, long start, long len, IgfsBlockLocation... locations) {
+        assert locations != null && locations.length > 0;
+
+        IgfsPath path = new IgfsPath(file);
+
+        Block block = new Block(path, start, len);
+
+        Collection<IgfsBlockLocation> locationsList = new ArrayList<>();
+
+        Collections.addAll(locationsList, locations);
+
+        BLOCK_MAP.put(block, locationsList);
+    }
+
+    /**
+     * Block.
+     */
+    private static class Block {
+        /** */
+        private final IgfsPath path;
+
+        /** */
+        private final long start;
+
+        /** */
+        private final long len;
+
+        /**
+         * Constructor.
+         *
+         * @param path Path.
+         * @param start Start.
+         * @param len Length.
+         */
+        private Block(IgfsPath path, long start, long len) {
+            this.path = path;
+            this.start = start;
+            this.len = len;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("RedundantIfStatement")
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (!(o instanceof Block)) return false;
+
+            Block block = (Block) o;
+
+            if (len != block.len)
+                return false;
+
+            if (start != block.start)
+                return false;
+
+            if (!path.equals(block.path))
+                return false;
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = path.hashCode();
+
+            res = 31 * res + (int) (start ^ (start >>> 32));
+            res = 31 * res + (int) (len ^ (len >>> 32));
+
+            return res;
+        }
+    }
+
+    /**
+     * Mocked job.
+     */
+    private static class MockJob implements GridHadoopJob {
+        /** Reducers count. */
+        private final int reducers;
+
+        /** */
+        private Collection<GridHadoopInputSplit> splitList;
+
+        /**
+         * Constructor.
+         *
+         * @param reducers Reducers count.
+         * @param splitList Splits.
+         */
+        private MockJob(int reducers, Collection<GridHadoopInputSplit> splitList) {
+            this.reducers = reducers;
+            this.splitList = splitList;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridHadoopJobId id() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridHadoopJobInfo info() {
+            return new HadoopDefaultJobInfo() {
+                @Override public int reducers() {
+                    return reducers;
+                }
+            };
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<GridHadoopInputSplit> input() throws IgniteCheckedException {
+            return splitList;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void initialize(boolean external, UUID nodeId) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void dispose(boolean external) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cleanupStagingDirectory() {
+            // No-op.
+        }
+    }
+
+    /**
+     * Mocked IGFS.
+     */
+    private static class MockIgfs implements IgfsEx {
+        /** {@inheritDoc} */
+        @Override public boolean isProxy(URI path) {
+            return PROXY_MAP.containsKey(path) && PROXY_MAP.get(path);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) {
+            return BLOCK_MAP.get(new Block(path, start, len));
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len,
+            long maxLen) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsContext context() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsPaths proxyPaths() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsInputStreamAdapter open(IgfsPath path) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsStatus globalSpace() throws IgniteCheckedException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void globalSampling(@Nullable Boolean val) throws IgniteCheckedException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Boolean globalSampling() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsLocalMetrics localMetrics() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long groupBlockSize() {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public String clientLogDirectory() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void clientLogDirectory(String logDir) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean evictExclude(IgfsPath path, boolean primary) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public String name() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsConfiguration configuration() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean exists(IgfsPath path) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public IgfsFile info(IgfsPath path) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsPathSummary summary(IgfsPath path) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rename(IgfsPath src, IgfsPath dest) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean delete(IgfsPath path, boolean recursive) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void mkdirs(IgfsPath path) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long usedSpaceSize() {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<String, String> properties() {
+            return Collections.emptyMap();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsOutputStream create(IgfsPath path, boolean overwrite) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
+            long blockSize, @Nullable Map<String, String> props) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite,
+            @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsOutputStream append(IgfsPath path, boolean create) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create,
+            @Nullable Map<String, String> props) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setTimes(IgfsPath path, long accessTime, long modificationTime) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgfsMetrics metrics() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resetMetrics() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public long size(IgfsPath path) {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void format() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
+            Collection<IgfsPath> paths, @Nullable T arg) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
+            Collection<IgfsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
+            @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, @Nullable T arg) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
+            @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, boolean skipNonExistentFiles,
+            long maxRangeLen, @Nullable T arg) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteUuid nextAffinityKey() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteFs withAsync() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isAsync() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <R> IgniteFuture<R> future() {
+            return null;
+        }
+    }
+
+    /**
+     * Mocked Grid.
+     */
+    @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
+    private static class MockIgnite extends IgniteSpringBean implements IgniteEx {
+        /** {@inheritDoc} */
+        @Override public IgniteClusterEx cluster() {
+            return (IgniteClusterEx)super.cluster();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteFs igfsx(String name) {
+            assert F.eq("igfs", name);
+
+            return IGFS;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridHadoop hadoop() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String name() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public <K extends GridCacheUtilityKey, V> GridCacheProjectionEx<K, V> utilityCache(Class<K> keyCls,
+            Class<V> valCls) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public <K, V> GridCache<K, V> cachex(@Nullable String name) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public <K, V> GridCache<K, V> cachex() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public Collection<GridCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super GridCache<?, ?>>... p) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean eventUserRecordable(int type) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean allEventsUserRecordable(int[] types) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Collection<String> compatibleVersions() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isJmxRemoteEnabled() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isRestartEnabled() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public ClusterNode localNode() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String latestVersion() {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimapSelftest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimapSelftest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimapSelftest.java
index 88dfd2b..8b5d2b6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimapSelftest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopConcurrentHashMultimapSelftest.java
@@ -54,9 +54,9 @@ public class GridHadoopConcurrentHashMultimapSelftest extends GridHadoopAbstract
 
         GridHadoopTaskContext taskCtx = new TaskContext();
 
-        GridHadoopConcurrentHashMultimap m = new GridHadoopConcurrentHashMultimap(job, mem, mapSize);
+        HadoopConcurrentHashMultimap m = new HadoopConcurrentHashMultimap(job, mem, mapSize);
 
-        GridHadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx);
+        HadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx);
 
         Multimap<Integer, Integer> mm = ArrayListMultimap.create();
         Multimap<Integer, Integer> vis = ArrayListMultimap.create();
@@ -90,7 +90,7 @@ public class GridHadoopConcurrentHashMultimapSelftest extends GridHadoopAbstract
         assertEquals(0, mem.allocatedSize());
     }
 
-    private void check(GridHadoopConcurrentHashMultimap m, Multimap<Integer, Integer> mm,
+    private void check(HadoopConcurrentHashMultimap m, Multimap<Integer, Integer> mm,
         final Multimap<Integer, Integer> vis, GridHadoopTaskContext taskCtx) throws Exception {
         final GridHadoopTaskInput in = m.input(taskCtx);
 
@@ -129,7 +129,7 @@ public class GridHadoopConcurrentHashMultimapSelftest extends GridHadoopAbstract
 
         final GridDataInput dataInput = new GridUnsafeDataInput();
 
-        m.visit(false, new GridHadoopConcurrentHashMultimap.Visitor() {
+        m.visit(false, new HadoopConcurrentHashMultimap.Visitor() {
             /** */
             IntWritable key = new IntWritable();
 
@@ -184,7 +184,7 @@ public class GridHadoopConcurrentHashMultimapSelftest extends GridHadoopAbstract
 
             final GridHadoopTaskContext taskCtx = new TaskContext();
 
-            final GridHadoopConcurrentHashMultimap m = new GridHadoopConcurrentHashMultimap(job, mem, 16);
+            final HadoopConcurrentHashMultimap m = new HadoopConcurrentHashMultimap(job, mem, 16);
 
             final ConcurrentMap<Integer, Collection<Integer>> mm = new ConcurrentHashMap<>();
 
@@ -199,7 +199,7 @@ public class GridHadoopConcurrentHashMultimapSelftest extends GridHadoopAbstract
                     IntWritable key = new IntWritable();
                     IntWritable val = new IntWritable();
 
-                    GridHadoopMultimap.Adder a = m.startAdding(taskCtx);
+                    HadoopMultimap.Adder a = m.startAdding(taskCtx);
 
                     for (int i = 0; i < 50000; i++) {
                         int k = rnd.nextInt(32000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMapSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMapSelfTest.java
index 92177ad..90d957b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMapSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMapSelfTest.java
@@ -89,9 +89,9 @@ public class GridHadoopHashMapSelfTest extends GridHadoopAbstractMapTest {
 
         GridHadoopTaskContext taskCtx = new TaskContext();
 
-        final GridHadoopHashMultimap m = new GridHadoopHashMultimap(new JobInfo(), mem, mapSize);
+        final HadoopHashMultimap m = new HadoopHashMultimap(new JobInfo(), mem, mapSize);
 
-        GridHadoopMultimap.Adder a = m.startAdding(taskCtx);
+        HadoopMultimap.Adder a = m.startAdding(taskCtx);
 
         Multimap<Integer, Integer> mm = ArrayListMultimap.create();
 
@@ -124,7 +124,7 @@ public class GridHadoopHashMapSelfTest extends GridHadoopAbstractMapTest {
         assertEquals(0, mem.allocatedSize());
     }
 
-    private void check(GridHadoopHashMultimap m, Multimap<Integer, Integer> mm, GridHadoopTaskContext taskCtx) throws Exception {
+    private void check(HadoopHashMultimap m, Multimap<Integer, Integer> mm, GridHadoopTaskContext taskCtx) throws Exception {
         final GridHadoopTaskInput in = m.input(taskCtx);
 
         Map<Integer, Collection<Integer>> mmm = mm.asMap();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipListSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipListSelfTest.java
index 6ba00ad..3d930ff 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipListSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipListSelfTest.java
@@ -47,7 +47,7 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
         int all = 10000;
 
         for (int i = 0; i < all; i++) {
-            int level = GridHadoopSkipList.randomLevel(rnd);
+            int level = HadoopSkipList.randomLevel(rnd);
 
             levelsCnts[level]++;
         }
@@ -86,9 +86,9 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
 
         GridHadoopTaskContext taskCtx = new TaskContext();
 
-        GridHadoopMultimap m = new GridHadoopSkipList(job, mem);
+        HadoopMultimap m = new HadoopSkipList(job, mem);
 
-        GridHadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx);
+        HadoopConcurrentHashMultimap.Adder a = m.startAdding(taskCtx);
 
         Multimap<Integer, Integer> mm = ArrayListMultimap.create();
         Multimap<Integer, Integer> vis = ArrayListMultimap.create();
@@ -122,7 +122,7 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
         assertEquals(0, mem.allocatedSize());
     }
 
-    private void check(GridHadoopMultimap m, Multimap<Integer, Integer> mm, final Multimap<Integer, Integer> vis, GridHadoopTaskContext taskCtx)
+    private void check(HadoopMultimap m, Multimap<Integer, Integer> mm, final Multimap<Integer, Integer> vis, GridHadoopTaskContext taskCtx)
         throws Exception {
         final GridHadoopTaskInput in = m.input(taskCtx);
 
@@ -165,7 +165,7 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
 
         final GridDataInput dataInput = new GridUnsafeDataInput();
 
-        m.visit(false, new GridHadoopConcurrentHashMultimap.Visitor() {
+        m.visit(false, new HadoopConcurrentHashMultimap.Visitor() {
             /** */
             IntWritable key = new IntWritable();
 
@@ -220,7 +220,7 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
 
             final GridHadoopTaskContext taskCtx = new TaskContext();
 
-            final GridHadoopMultimap m = new GridHadoopSkipList(job, mem);
+            final HadoopMultimap m = new HadoopSkipList(job, mem);
 
             final ConcurrentMap<Integer, Collection<Integer>> mm = new ConcurrentHashMap<>();
 
@@ -235,7 +235,7 @@ public class GridHadoopSkipListSelfTest extends GridHadoopAbstractMapTest {
                     IntWritable key = new IntWritable();
                     IntWritable val = new IntWritable();
 
-                    GridHadoopMultimap.Adder a = m.startAdding(taskCtx);
+                    HadoopMultimap.Adder a = m.startAdding(taskCtx);
 
                     for (int i = 0; i < 50000; i++) {
                         int k = rnd.nextInt(32000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java
index 39a537b..98475fb 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/streams/GridHadoopDataStreamSelfTest.java
@@ -31,7 +31,7 @@ public class GridHadoopDataStreamSelfTest extends GridCommonAbstractTest {
     public void testStreams() throws IOException {
         GridUnsafeMemory mem = new GridUnsafeMemory(0);
 
-        GridHadoopDataOutStream out = new GridHadoopDataOutStream(mem);
+        HadoopDataOutStream out = new HadoopDataOutStream(mem);
 
         int size = 4 * 1024;
 
@@ -86,7 +86,7 @@ public class GridHadoopDataStreamSelfTest extends GridCommonAbstractTest {
         out.write(new byte[]{0,1,2,3}, 1, 2);
         out.writeUTF("mom washes rum");
 
-        GridHadoopDataInStream in = new GridHadoopDataInStream(mem);
+        HadoopDataInStream in = new HadoopDataInStream(mem);
 
         in.buffer().set(ptr, out.buffer().pointer());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
deleted file mode 100644
index c97b6ab..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
+++ /dev/null
@@ -1,119 +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.taskexecutor;
-
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.jdk8.backport.*;
-
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- *
- */
-public class GridHadoopExecutorServiceTest extends GridCommonAbstractTest {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testExecutesAll() throws Exception {
-        final GridHadoopExecutorService exec = new GridHadoopExecutorService(log, "_GRID_NAME_", 10, 5);
-
-        for (int i = 0; i < 5; i++) {
-            final int loops = 5000;
-            int threads = 17;
-
-            final LongAdder sum = new LongAdder();
-
-            multithreaded(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    for (int i = 0; i < loops; i++) {
-                        exec.submit(new Callable<Void>() {
-                            @Override
-                            public Void call() throws Exception {
-                                sum.increment();
-
-                                return null;
-                            }
-                        });
-                    }
-
-                    return null;
-                }
-            }, threads);
-
-            while (exec.active() != 0) {
-                X.println("__ active: " + exec.active());
-
-                Thread.sleep(200);
-            }
-
-            assertEquals(threads * loops, sum.sum());
-
-            X.println("_ ok");
-        }
-
-        assertTrue(exec.shutdown(0));
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testShutdown() throws Exception {
-        for (int i = 0; i < 5; i++) {
-            final GridHadoopExecutorService exec = new GridHadoopExecutorService(log, "_GRID_NAME_", 10, 5);
-
-            final LongAdder sum = new LongAdder();
-
-            final AtomicBoolean finish = new AtomicBoolean();
-
-            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    while (!finish.get()) {
-                        exec.submit(new Callable<Void>() {
-                            @Override public Void call() throws Exception {
-                                sum.increment();
-
-                                return null;
-                            }
-                        });
-                    }
-
-                    return null;
-                }
-            }, 19);
-
-            Thread.sleep(200);
-
-            assertTrue(exec.shutdown(50));
-
-            long res = sum.sum();
-
-            assertTrue(res > 0);
-
-            finish.set(true);
-
-            fut.get();
-
-            assertEquals(res, sum.sum()); // Nothing was executed after shutdown.
-
-            X.println("_ ok");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java
new file mode 100644
index 0000000..aa50fa9
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorServiceTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jdk8.backport.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class HadoopExecutorServiceTest extends GridCommonAbstractTest {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExecutesAll() throws Exception {
+        final HadoopExecutorService exec = new HadoopExecutorService(log, "_GRID_NAME_", 10, 5);
+
+        for (int i = 0; i < 5; i++) {
+            final int loops = 5000;
+            int threads = 17;
+
+            final LongAdder sum = new LongAdder();
+
+            multithreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    for (int i = 0; i < loops; i++) {
+                        exec.submit(new Callable<Void>() {
+                            @Override
+                            public Void call() throws Exception {
+                                sum.increment();
+
+                                return null;
+                            }
+                        });
+                    }
+
+                    return null;
+                }
+            }, threads);
+
+            while (exec.active() != 0) {
+                X.println("__ active: " + exec.active());
+
+                Thread.sleep(200);
+            }
+
+            assertEquals(threads * loops, sum.sum());
+
+            X.println("_ ok");
+        }
+
+        assertTrue(exec.shutdown(0));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShutdown() throws Exception {
+        for (int i = 0; i < 5; i++) {
+            final HadoopExecutorService exec = new HadoopExecutorService(log, "_GRID_NAME_", 10, 5);
+
+            final LongAdder sum = new LongAdder();
+
+            final AtomicBoolean finish = new AtomicBoolean();
+
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    while (!finish.get()) {
+                        exec.submit(new Callable<Void>() {
+                            @Override public Void call() throws Exception {
+                                sum.increment();
+
+                                return null;
+                            }
+                        });
+                    }
+
+                    return null;
+                }
+            }, 19);
+
+            Thread.sleep(200);
+
+            assertTrue(exec.shutdown(50));
+
+            long res = sum.sum();
+
+            assertTrue(res > 0);
+
+            finish.set(true);
+
+            fut.get();
+
+            assertEquals(res, sum.sum()); // Nothing was executed after shutdown.
+
+            X.println("_ ok");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
index a725ddc..dd3c5d4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunicationSelfTest.java
@@ -125,7 +125,7 @@ public class GridHadoopExternalCommunicationSelfTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public void onMessageReceived(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
+        @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
             assert msg instanceof TestMessage;
 
             msgs.add((TestMessage)msg);
@@ -134,7 +134,7 @@ public class GridHadoopExternalCommunicationSelfTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
-        @Override public void onConnectionLost(GridHadoopProcessDescriptor desc) {
+        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
             // No-op.
         }
 
@@ -157,7 +157,7 @@ public class GridHadoopExternalCommunicationSelfTest extends GridCommonAbstractT
     /**
      *
      */
-    private static class TestMessage implements GridHadoopMessage {
+    private static class TestMessage implements HadoopMessage {
         /** From index. */
         private int from;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index f599ed9..822ab8f 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -83,7 +83,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopValidationSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(GridHadoopDefaultMapReducePlannerSelfTest.class.getName())));
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopDefaultMapReducePlannerSelfTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopJobTrackerSelfTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(GridHadoopHashMapSelfTest.class.getName())));


[04/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ace354c6/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
deleted file mode 100644
index 0beaf32..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
+++ /dev/null
@@ -1,1625 +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.jobtracker;
-
-import org.apache.ignite.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.events.EventType;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.managers.eventstorage.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.event.*;
-import javax.cache.expiry.*;
-import javax.cache.processor.*;
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
-import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
-
-/**
- * Hadoop job tracker.
- */
-public class GridHadoopJobTracker extends GridHadoopComponent {
-    /** */
-    private final GridMutex mux = new GridMutex();
-
-    /** */
-    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaPrj;
-
-    /** Projection with expiry policy for finished job updates. */
-    private volatile GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaPrj;
-
-    /** Map-reduce execution planner. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private GridHadoopMapReducePlanner mrPlanner;
-
-    /** All the known jobs. */
-    private final ConcurrentMap<GridHadoopJobId, GridFutureAdapterEx<GridHadoopJob>> jobs = new ConcurrentHashMap8<>();
-
-    /** Locally active jobs. */
-    private final ConcurrentMap<GridHadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
-
-    /** Locally requested finish futures. */
-    private final ConcurrentMap<GridHadoopJobId, GridFutureAdapter<GridHadoopJobId>> activeFinishFuts =
-        new ConcurrentHashMap8<>();
-
-    /** Event processing service. */
-    private ExecutorService evtProcSvc;
-
-    /** Component busy lock. */
-    private GridSpinReadWriteLock busyLock;
-
-    /** Closure to check result of async transform of system cache. */
-    private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
-        @Override public void apply(IgniteInternalFuture<?> gridFut) {
-            try {
-                gridFut.get();
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to transform system cache.", e);
-            }
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public void start(GridHadoopContext ctx) throws IgniteCheckedException {
-        super.start(ctx);
-
-        busyLock = new GridSpinReadWriteLock();
-
-        evtProcSvc = Executors.newFixedThreadPool(1);
-    }
-
-    /**
-     * @return Job meta projection.
-     */
-    @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> jobMetaCache() {
-        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = jobMetaPrj;
-
-        if (prj == null) {
-            synchronized (mux) {
-                if ((prj = jobMetaPrj) == null) {
-                    CacheProjection<Object, Object> sysCache = ctx.kernalContext().cache()
-                        .cache(CU.SYS_CACHE_HADOOP_MR);
-
-                    assert sysCache != null;
-
-                    mrPlanner = ctx.planner();
-
-                    try {
-                        ctx.kernalContext().resource().injectGeneric(mrPlanner);
-                    }
-                    catch (IgniteCheckedException e) { // Must not happen.
-                        U.error(log, "Failed to inject resources.", e);
-
-                        throw new IllegalStateException(e);
-                    }
-
-                    jobMetaPrj = prj = (GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata>)
-                        sysCache.projection(GridHadoopJobId.class, GridHadoopJobMetadata.class);
-
-                    if (ctx.configuration().getFinishedJobInfoTtl() > 0) {
-                        ExpiryPolicy finishedJobPlc = new ModifiedExpiryPolicy(
-                            new Duration(MILLISECONDS, ctx.configuration().getFinishedJobInfoTtl()));
-
-                        finishedJobMetaPrj = prj.withExpiryPolicy(finishedJobPlc);
-                    }
-                    else
-                        finishedJobMetaPrj = jobMetaPrj;
-                }
-            }
-        }
-
-        return prj;
-    }
-
-    /**
-     * @return Projection with expiry policy for finished job updates.
-     */
-    private GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> finishedJobMetaCache() {
-        GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> prj = finishedJobMetaPrj;
-
-        if (prj == null) {
-            jobMetaCache();
-
-            prj = finishedJobMetaPrj;
-
-            assert prj != null;
-        }
-
-        return prj;
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("deprecation")
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
-
-        jobMetaCache().context().continuousQueries().executeInternalQuery(
-            new CacheEntryUpdatedListener<GridHadoopJobId, GridHadoopJobMetadata>() {
-                @Override public void onUpdated(final Iterable<CacheEntryEvent<? extends GridHadoopJobId,
-                    ? extends GridHadoopJobMetadata>> evts) {
-                    if (!busyLock.tryReadLock())
-                        return;
-
-                    try {
-                        // Must process query callback in a separate thread to avoid deadlocks.
-                        evtProcSvc.submit(new EventHandler() {
-                            @Override protected void body() throws IgniteCheckedException {
-                                processJobMetadataUpdates(evts);
-                            }
-                        });
-                    }
-                    finally {
-                        busyLock.readUnlock();
-                    }
-                }
-            },
-            null,
-            true,
-            true
-        );
-
-        ctx.kernalContext().event().addLocalEventListener(new GridLocalEventListener() {
-            @Override public void onEvent(final Event evt) {
-                if (!busyLock.tryReadLock())
-                    return;
-
-                try {
-                    // Must process discovery callback in a separate thread to avoid deadlock.
-                    evtProcSvc.submit(new EventHandler() {
-                        @Override protected void body() {
-                            processNodeLeft((DiscoveryEvent)evt);
-                        }
-                    });
-                }
-                finally {
-                    busyLock.readUnlock();
-                }
-            }
-        }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        super.onKernalStop(cancel);
-
-        busyLock.writeLock();
-
-        evtProcSvc.shutdown();
-
-        // Fail all pending futures.
-        for (GridFutureAdapter<GridHadoopJobId> fut : activeFinishFuts.values())
-            fut.onDone(new IgniteCheckedException("Failed to execute Hadoop map-reduce job (grid is stopping)."));
-    }
-
-    /**
-     * Submits execution of Hadoop job to grid.
-     *
-     * @param jobId Job ID.
-     * @param info Job info.
-     * @return Job completion future.
-     */
-    @SuppressWarnings("unchecked")
-    public IgniteInternalFuture<GridHadoopJobId> submit(GridHadoopJobId jobId, GridHadoopJobInfo info) {
-        if (!busyLock.tryReadLock()) {
-            return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to execute map-reduce job " +
-                "(grid is stopping): " + info));
-        }
-
-        try {
-            long jobPrepare = U.currentTimeMillis();
-
-            if (jobs.containsKey(jobId) || jobMetaCache().containsKey(jobId))
-                throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
-
-            GridHadoopJob job = job(jobId, info);
-
-            GridHadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
-
-            GridHadoopJobMetadata meta = new GridHadoopJobMetadata(ctx.localNodeId(), jobId, info);
-
-            meta.mapReducePlan(mrPlan);
-
-            meta.pendingSplits(allSplits(mrPlan));
-            meta.pendingReducers(allReducers(mrPlan));
-
-            GridFutureAdapter<GridHadoopJobId> completeFut = new GridFutureAdapter<>();
-
-            GridFutureAdapter<GridHadoopJobId> old = activeFinishFuts.put(jobId, completeFut);
-
-            assert old == null : "Duplicate completion future [jobId=" + jobId + ", old=" + old + ']';
-
-            if (log.isDebugEnabled())
-                log.debug("Submitting job metadata [jobId=" + jobId + ", meta=" + meta + ']');
-
-            long jobStart = U.currentTimeMillis();
-
-            GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(meta.counters(),
-                ctx.localNodeId());
-
-            perfCntr.clientSubmissionEvents(info);
-            perfCntr.onJobPrepare(jobPrepare);
-            perfCntr.onJobStart(jobStart);
-
-            if (jobMetaCache().putIfAbsent(jobId, meta) != null)
-                throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
-
-            return completeFut;
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to submit job: " + jobId, e);
-
-            return new GridFinishedFutureEx<>(e);
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * Convert Hadoop job metadata to job status.
-     *
-     * @param meta Metadata.
-     * @return Status.
-     */
-    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
-    public static GridHadoopJobStatus status(GridHadoopJobMetadata meta) {
-        GridHadoopJobInfo jobInfo = meta.jobInfo();
-
-        return new GridHadoopJobStatus(
-            meta.jobId(),
-            jobInfo.jobName(),
-            jobInfo.user(),
-            meta.pendingSplits() != null ? meta.pendingSplits().size() : 0,
-            meta.pendingReducers() != null ? meta.pendingReducers().size() : 0,
-            meta.mapReducePlan().mappers(),
-            meta.mapReducePlan().reducers(),
-            meta.phase(),
-            meta.failCause() != null,
-            meta.version()
-        );
-    }
-
-    /**
-     * Gets hadoop job status for given job ID.
-     *
-     * @param jobId Job ID to get status for.
-     * @return Job status for given job ID or {@code null} if job was not found.
-     */
-    @Nullable public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            return null; // Grid is stopping.
-
-        try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-            return meta != null ? status(meta) : null;
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * Gets job finish future.
-     *
-     * @param jobId Job ID.
-     * @return Finish future or {@code null}.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            return null; // Grid is stopping.
-
-        try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-            if (meta == null)
-                return null;
-
-            if (log.isTraceEnabled())
-                log.trace("Got job metadata for status check [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
-
-            if (meta.phase() == PHASE_COMPLETE) {
-                if (log.isTraceEnabled())
-                    log.trace("Job is complete, returning finished future: " + jobId);
-
-                return new GridFinishedFutureEx<>(jobId, meta.failCause());
-            }
-
-            GridFutureAdapter<GridHadoopJobId> fut = F.addIfAbsent(activeFinishFuts, jobId,
-                new GridFutureAdapter<GridHadoopJobId>());
-
-            // Get meta from cache one more time to close the window.
-            meta = jobMetaCache().get(jobId);
-
-            if (log.isTraceEnabled())
-                log.trace("Re-checking job metadata [locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');
-
-            if (meta == null) {
-                fut.onDone();
-
-                activeFinishFuts.remove(jobId , fut);
-            }
-            else if (meta.phase() == PHASE_COMPLETE) {
-                fut.onDone(jobId, meta.failCause());
-
-                activeFinishFuts.remove(jobId , fut);
-            }
-
-            return fut;
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * Gets job plan by job ID.
-     *
-     * @param jobId Job ID.
-     * @return Job plan.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridHadoopMapReducePlan plan(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            return null;
-
-        try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-            if (meta != null)
-                return meta.mapReducePlan();
-
-            return null;
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * Callback from task executor invoked when a task has been finished.
-     *
-     * @param info Task info.
-     * @param status Task status.
-     */
-    @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
-    public void onTaskFinished(GridHadoopTaskInfo info, GridHadoopTaskStatus status) {
-        if (!busyLock.tryReadLock())
-            return;
-
-        try {
-            assert status.state() != RUNNING;
-
-            if (log.isDebugEnabled())
-                log.debug("Received task finished callback [info=" + info + ", status=" + status + ']');
-
-            JobLocalState state = activeJobs.get(info.jobId());
-
-            // Task CRASHes with null fail cause.
-            assert (status.state() != FAILED) || status.failCause() != null :
-                "Invalid task status [info=" + info + ", status=" + status + ']';
-
-            assert state != null || (ctx.jobUpdateLeader() && (info.type() == COMMIT || info.type() == ABORT)):
-                "Missing local state for finished task [info=" + info + ", status=" + status + ']';
-
-            StackedProcessor incrCntrs = null;
-
-            if (status.state() == COMPLETED)
-                incrCntrs = new IncrementCountersProcessor(null, status.counters());
-
-            switch (info.type()) {
-                case SETUP: {
-                    state.onSetupFinished(info, status, incrCntrs);
-
-                    break;
-                }
-
-                case MAP: {
-                    state.onMapFinished(info, status, incrCntrs);
-
-                    break;
-                }
-
-                case REDUCE: {
-                    state.onReduceFinished(info, status, incrCntrs);
-
-                    break;
-                }
-
-                case COMBINE: {
-                    state.onCombineFinished(info, status, incrCntrs);
-
-                    break;
-                }
-
-                case COMMIT:
-                case ABORT: {
-                    GridCacheProjectionEx<GridHadoopJobId, GridHadoopJobMetadata> cache = finishedJobMetaCache();
-
-                    cache.invokeAsync(info.jobId(), new UpdatePhaseProcessor(incrCntrs, PHASE_COMPLETE)).
-                        listenAsync(failsLog);
-
-                    break;
-                }
-            }
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * @param jobId Job id.
-     * @param c Closure of operation.
-     */
-    private void transform(GridHadoopJobId jobId, EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void> c) {
-        jobMetaCache().invokeAsync(jobId, c).listenAsync(failsLog);
-    }
-
-    /**
-     * Callback from task executor called when process is ready to received shuffle messages.
-     *
-     * @param jobId Job ID.
-     * @param reducers Reducers.
-     * @param desc Process descriptor.
-     */
-    public void onExternalMappersInitialized(GridHadoopJobId jobId, Collection<Integer> reducers,
-        GridHadoopProcessDescriptor desc) {
-        transform(jobId, new InitializeReducersProcessor(null, reducers, desc));
-    }
-
-    /**
-     * Gets all input splits for given hadoop map-reduce plan.
-     *
-     * @param plan Map-reduce plan.
-     * @return Collection of all input splits that should be processed.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private Map<GridHadoopInputSplit, Integer> allSplits(GridHadoopMapReducePlan plan) {
-        Map<GridHadoopInputSplit, Integer> res = new HashMap<>();
-
-        int taskNum = 0;
-
-        for (UUID nodeId : plan.mapperNodeIds()) {
-            for (GridHadoopInputSplit split : plan.mappers(nodeId)) {
-                if (res.put(split, taskNum++) != null)
-                    throw new IllegalStateException("Split duplicate.");
-            }
-        }
-
-        return res;
-    }
-
-    /**
-     * Gets all reducers for this job.
-     *
-     * @param plan Map-reduce plan.
-     * @return Collection of reducers.
-     */
-    private Collection<Integer> allReducers(GridHadoopMapReducePlan plan) {
-        Collection<Integer> res = new HashSet<>();
-
-        for (int i = 0; i < plan.reducers(); i++)
-            res.add(i);
-
-        return res;
-    }
-
-    /**
-     * Processes node leave (or fail) event.
-     *
-     * @param evt Discovery event.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private void processNodeLeft(DiscoveryEvent evt) {
-        if (log.isDebugEnabled())
-            log.debug("Processing discovery event [locNodeId=" + ctx.localNodeId() + ", evt=" + evt + ']');
-
-        // Check only if this node is responsible for job status updates.
-        if (ctx.jobUpdateLeader()) {
-            boolean checkSetup = evt.eventNode().order() < ctx.localNodeOrder();
-
-            // Iteration over all local entries is correct since system cache is REPLICATED.
-            for (Object metaObj : jobMetaCache().values()) {
-                GridHadoopJobMetadata meta = (GridHadoopJobMetadata)metaObj;
-
-                GridHadoopJobId jobId = meta.jobId();
-
-                GridHadoopMapReducePlan plan = meta.mapReducePlan();
-
-                GridHadoopJobPhase phase = meta.phase();
-
-                try {
-                    if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
-                        // Failover setup task.
-                        GridHadoopJob job = job(jobId, meta.jobInfo());
-
-                        Collection<GridHadoopTaskInfo> setupTask = setupTask(jobId);
-
-                        assert setupTask != null;
-
-                        ctx.taskExecutor().run(job, setupTask);
-                    }
-                    else if (phase == PHASE_MAP || phase == PHASE_REDUCE) {
-                        // Must check all nodes, even that are not event node ID due to
-                        // multiple node failure possibility.
-                        Collection<GridHadoopInputSplit> cancelSplits = null;
-
-                        for (UUID nodeId : plan.mapperNodeIds()) {
-                            if (ctx.kernalContext().discovery().node(nodeId) == null) {
-                                // Node has left the grid.
-                                Collection<GridHadoopInputSplit> mappers = plan.mappers(nodeId);
-
-                                if (cancelSplits == null)
-                                    cancelSplits = new HashSet<>();
-
-                                cancelSplits.addAll(mappers);
-                            }
-                        }
-
-                        Collection<Integer> cancelReducers = null;
-
-                        for (UUID nodeId : plan.reducerNodeIds()) {
-                            if (ctx.kernalContext().discovery().node(nodeId) == null) {
-                                // Node has left the grid.
-                                int[] reducers = plan.reducers(nodeId);
-
-                                if (cancelReducers == null)
-                                    cancelReducers = new HashSet<>();
-
-                                for (int rdc : reducers)
-                                    cancelReducers.add(rdc);
-                            }
-                        }
-
-                        if (cancelSplits != null || cancelReducers != null)
-                            jobMetaCache().invoke(meta.jobId(), new CancelJobProcessor(null, new IgniteCheckedException(
-                                "One or more nodes participating in map-reduce job execution failed."), cancelSplits,
-                                cancelReducers));
-                    }
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to cancel job: " + meta, e);
-                }
-            }
-        }
-    }
-
-    /**
-     * @param updated Updated cache entries.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void processJobMetadataUpdates(
-        Iterable<CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata>> updated)
-        throws IgniteCheckedException {
-        UUID locNodeId = ctx.localNodeId();
-
-        for (CacheEntryEvent<? extends GridHadoopJobId, ? extends GridHadoopJobMetadata> entry : updated) {
-            GridHadoopJobId jobId = entry.getKey();
-            GridHadoopJobMetadata meta = entry.getValue();
-
-            if (meta == null || !ctx.isParticipating(meta))
-                continue;
-
-            if (log.isDebugEnabled())
-                log.debug("Processing job metadata update callback [locNodeId=" + locNodeId +
-                    ", meta=" + meta + ']');
-
-            try {
-                ctx.taskExecutor().onJobStateChanged(meta);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to process job state changed callback (will fail the job) " +
-                    "[locNodeId=" + locNodeId + ", jobId=" + jobId + ", meta=" + meta + ']', e);
-
-                transform(jobId, new CancelJobProcessor(null, e));
-
-                continue;
-            }
-
-            processJobMetaUpdate(jobId, meta, locNodeId);
-        }
-    }
-
-    /**
-     * @param jobId  Job ID.
-     * @param plan Map-reduce plan.
-     */
-    private void printPlan(GridHadoopJobId jobId, GridHadoopMapReducePlan plan) {
-        log.info("Plan for " + jobId);
-
-        SB b = new SB();
-
-        b.a("   Map: ");
-
-        for (UUID nodeId : plan.mapperNodeIds())
-            b.a(nodeId).a("=").a(plan.mappers(nodeId).size()).a(' ');
-
-        log.info(b.toString());
-
-        b = new SB();
-
-        b.a("   Reduce: ");
-
-        for (UUID nodeId : plan.reducerNodeIds())
-            b.a(nodeId).a("=").a(Arrays.toString(plan.reducers(nodeId))).a(' ');
-
-        log.info(b.toString());
-    }
-
-    /**
-     * @param jobId Job ID.
-     * @param meta Job metadata.
-     * @param locNodeId Local node ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void processJobMetaUpdate(GridHadoopJobId jobId, GridHadoopJobMetadata meta, UUID locNodeId)
-        throws IgniteCheckedException {
-        JobLocalState state = activeJobs.get(jobId);
-
-        GridHadoopJob job = job(jobId, meta.jobInfo());
-
-        GridHadoopMapReducePlan plan = meta.mapReducePlan();
-
-        switch (meta.phase()) {
-            case PHASE_SETUP: {
-                if (ctx.jobUpdateLeader()) {
-                    Collection<GridHadoopTaskInfo> setupTask = setupTask(jobId);
-
-                    if (setupTask != null)
-                        ctx.taskExecutor().run(job, setupTask);
-                }
-
-                break;
-            }
-
-            case PHASE_MAP: {
-                // Check if we should initiate new task on local node.
-                Collection<GridHadoopTaskInfo> tasks = mapperTasks(plan.mappers(locNodeId), meta);
-
-                if (tasks != null)
-                    ctx.taskExecutor().run(job, tasks);
-
-                break;
-            }
-
-            case PHASE_REDUCE: {
-                if (meta.pendingReducers().isEmpty() && ctx.jobUpdateLeader()) {
-                    GridHadoopTaskInfo info = new GridHadoopTaskInfo(COMMIT, jobId, 0, 0, null);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Submitting COMMIT task for execution [locNodeId=" + locNodeId +
-                                ", jobId=" + jobId + ']');
-
-                    ctx.taskExecutor().run(job, Collections.singletonList(info));
-
-                    break;
-                }
-
-                Collection<GridHadoopTaskInfo> tasks = reducerTasks(plan.reducers(locNodeId), job);
-
-                if (tasks != null)
-                    ctx.taskExecutor().run(job, tasks);
-
-                break;
-            }
-
-            case PHASE_CANCELLING: {
-                // Prevent multiple task executor notification.
-                if (state != null && state.onCancel()) {
-                    if (log.isDebugEnabled())
-                        log.debug("Cancelling local task execution for job: " + meta);
-
-                    ctx.taskExecutor().cancelTasks(jobId);
-                }
-
-                if (meta.pendingSplits().isEmpty() && meta.pendingReducers().isEmpty()) {
-                    if (ctx.jobUpdateLeader()) {
-                        if (state == null)
-                            state = initState(jobId);
-
-                        // Prevent running multiple abort tasks.
-                        if (state.onAborted()) {
-                            GridHadoopTaskInfo info = new GridHadoopTaskInfo(ABORT, jobId, 0, 0, null);
-
-                            if (log.isDebugEnabled())
-                                log.debug("Submitting ABORT task for execution [locNodeId=" + locNodeId +
-                                        ", jobId=" + jobId + ']');
-
-                            ctx.taskExecutor().run(job, Collections.singletonList(info));
-                        }
-                    }
-
-                    break;
-                }
-                else {
-                    // Check if there are unscheduled mappers or reducers.
-                    Collection<GridHadoopInputSplit> cancelMappers = new ArrayList<>();
-                    Collection<Integer> cancelReducers = new ArrayList<>();
-
-                    Collection<GridHadoopInputSplit> mappers = plan.mappers(ctx.localNodeId());
-
-                    if (mappers != null) {
-                        for (GridHadoopInputSplit b : mappers) {
-                            if (state == null || !state.mapperScheduled(b))
-                                cancelMappers.add(b);
-                        }
-                    }
-
-                    int[] rdc = plan.reducers(ctx.localNodeId());
-
-                    if (rdc != null) {
-                        for (int r : rdc) {
-                            if (state == null || !state.reducerScheduled(r))
-                                cancelReducers.add(r);
-                        }
-                    }
-
-                    if (!cancelMappers.isEmpty() || !cancelReducers.isEmpty())
-                        transform(jobId, new CancelJobProcessor(null, cancelMappers, cancelReducers));
-                }
-
-                break;
-            }
-
-            case PHASE_COMPLETE: {
-                if (log.isDebugEnabled())
-                    log.debug("Job execution is complete, will remove local state from active jobs " +
-                        "[jobId=" + jobId + ", meta=" + meta + ']');
-
-                if (state != null) {
-                    state = activeJobs.remove(jobId);
-
-                    assert state != null;
-
-                    ctx.shuffle().jobFinished(jobId);
-                }
-
-                GridFutureAdapter<GridHadoopJobId> finishFut = activeFinishFuts.remove(jobId);
-
-                if (finishFut != null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Completing job future [locNodeId=" + locNodeId + ", meta=" + meta + ']');
-
-                    finishFut.onDone(jobId, meta.failCause());
-                }
-
-                if (ctx.jobUpdateLeader())
-                    job.cleanupStagingDirectory();
-
-                jobs.remove(jobId);
-
-                job.dispose(false);
-
-                if (ctx.jobUpdateLeader()) {
-                    ClassLoader ldr = job.getClass().getClassLoader();
-
-                    try {
-                        String statWriterClsName = job.info().property(GridHadoopUtils.JOB_COUNTER_WRITER_PROPERTY);
-
-                        if (statWriterClsName != null) {
-                            Class<?> cls = ldr.loadClass(statWriterClsName);
-
-                            GridHadoopCounterWriter writer = (GridHadoopCounterWriter)cls.newInstance();
-
-                            GridHadoopCounters cntrs = meta.counters();
-
-                            writer.write(job.info(), jobId, cntrs);
-                        }
-                    }
-                    catch (Exception e) {
-                        log.error("Can't write statistic due to: ", e);
-                    }
-                }
-
-                break;
-            }
-
-            default:
-                throw new IllegalStateException("Unknown phase: " + meta.phase());
-        }
-    }
-
-    /**
-     * Creates setup task based on job information.
-     *
-     * @param jobId Job ID.
-     * @return Setup task wrapped in collection.
-     */
-    @Nullable private Collection<GridHadoopTaskInfo> setupTask(GridHadoopJobId jobId) {
-        if (activeJobs.containsKey(jobId))
-            return null;
-        else {
-            initState(jobId);
-
-            return Collections.singleton(new GridHadoopTaskInfo(SETUP, jobId, 0, 0, null));
-        }
-    }
-
-    /**
-     * Creates mapper tasks based on job information.
-     *
-     * @param mappers Mapper blocks.
-     * @param meta Job metadata.
-     * @return Collection of created task infos or {@code null} if no mapper tasks scheduled for local node.
-     */
-    private Collection<GridHadoopTaskInfo> mapperTasks(Iterable<GridHadoopInputSplit> mappers, GridHadoopJobMetadata meta) {
-        UUID locNodeId = ctx.localNodeId();
-        GridHadoopJobId jobId = meta.jobId();
-
-        JobLocalState state = activeJobs.get(jobId);
-
-        Collection<GridHadoopTaskInfo> tasks = null;
-
-        if (mappers != null) {
-            if (state == null)
-                state = initState(jobId);
-
-            for (GridHadoopInputSplit split : mappers) {
-                if (state.addMapper(split)) {
-                    if (log.isDebugEnabled())
-                        log.debug("Submitting MAP task for execution [locNodeId=" + locNodeId +
-                            ", split=" + split + ']');
-
-                    GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split);
-
-                    if (tasks == null)
-                        tasks = new ArrayList<>();
-
-                    tasks.add(taskInfo);
-                }
-            }
-        }
-
-        return tasks;
-    }
-
-    /**
-     * Creates reducer tasks based on job information.
-     *
-     * @param reducers Reducers (may be {@code null}).
-     * @param job Job instance.
-     * @return Collection of task infos.
-     */
-    private Collection<GridHadoopTaskInfo> reducerTasks(int[] reducers, GridHadoopJob job) {
-        UUID locNodeId = ctx.localNodeId();
-        GridHadoopJobId jobId = job.id();
-
-        JobLocalState state = activeJobs.get(jobId);
-
-        Collection<GridHadoopTaskInfo> tasks = null;
-
-        if (reducers != null) {
-            if (state == null)
-                state = initState(job.id());
-
-            for (int rdc : reducers) {
-                if (state.addReducer(rdc)) {
-                    if (log.isDebugEnabled())
-                        log.debug("Submitting REDUCE task for execution [locNodeId=" + locNodeId +
-                            ", rdc=" + rdc + ']');
-
-                    GridHadoopTaskInfo taskInfo = new GridHadoopTaskInfo(REDUCE, jobId, rdc, 0, null);
-
-                    if (tasks == null)
-                        tasks = new ArrayList<>();
-
-                    tasks.add(taskInfo);
-                }
-            }
-        }
-
-        return tasks;
-    }
-
-    /**
-     * Initializes local state for given job metadata.
-     *
-     * @param jobId Job ID.
-     * @return Local state.
-     */
-    private JobLocalState initState(GridHadoopJobId jobId) {
-        return F.addIfAbsent(activeJobs, jobId, new JobLocalState());
-    }
-
-    /**
-     * Gets or creates job instance.
-     *
-     * @param jobId Job ID.
-     * @param jobInfo Job info.
-     * @return Job.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public GridHadoopJob job(GridHadoopJobId jobId, @Nullable GridHadoopJobInfo jobInfo) throws IgniteCheckedException {
-        GridFutureAdapterEx<GridHadoopJob> fut = jobs.get(jobId);
-
-        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapterEx<GridHadoopJob>())) != null)
-            return fut.get();
-
-        fut = jobs.get(jobId);
-
-        GridHadoopJob job = null;
-
-        try {
-            if (jobInfo == null) {
-                GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-                if (meta == null)
-                    throw new IgniteCheckedException("Failed to find job metadata for ID: " + jobId);
-
-                jobInfo = meta.jobInfo();
-            }
-
-            job = jobInfo.createJob(jobId, log);
-
-            job.initialize(false, ctx.localNodeId());
-
-            fut.onDone(job);
-
-            return job;
-        }
-        catch (IgniteCheckedException e) {
-            fut.onDone(e);
-
-            jobs.remove(jobId, fut);
-
-            if (job != null) {
-                try {
-                    job.dispose(false);
-                }
-                catch (IgniteCheckedException e0) {
-                    U.error(log, "Failed to dispose job: " + jobId, e0);
-                }
-            }
-
-            throw e;
-        }
-    }
-
-    /**
-     * Kills job.
-     *
-     * @param jobId Job ID.
-     * @return {@code True} if job was killed.
-     * @throws IgniteCheckedException If failed.
-     */
-    public boolean killJob(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            return false; // Grid is stopping.
-
-        try {
-            GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-            if (meta != null && meta.phase() != PHASE_COMPLETE && meta.phase() != PHASE_CANCELLING) {
-                GridHadoopTaskCancelledException err = new GridHadoopTaskCancelledException("Job cancelled.");
-
-                jobMetaCache().invoke(jobId, new CancelJobProcessor(null, err));
-            }
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-
-        IgniteInternalFuture<?> fut = finishFuture(jobId);
-
-        if (fut != null) {
-            try {
-                fut.get();
-            }
-            catch (Throwable e) {
-                if (e.getCause() instanceof GridHadoopTaskCancelledException)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Returns job counters.
-     *
-     * @param jobId Job identifier.
-     * @return Job counters or {@code null} if job cannot be found.
-     * @throws IgniteCheckedException If failed.
-     */
-    @Nullable public GridHadoopCounters jobCounters(GridHadoopJobId jobId) throws IgniteCheckedException {
-        if (!busyLock.tryReadLock())
-            return null;
-
-        try {
-            final GridHadoopJobMetadata meta = jobMetaCache().get(jobId);
-
-            return meta != null ? meta.counters() : null;
-        }
-        finally {
-            busyLock.readUnlock();
-        }
-    }
-
-    /**
-     * Event handler protected by busy lock.
-     */
-    private abstract class EventHandler implements Runnable {
-        /** {@inheritDoc} */
-        @Override public void run() {
-            if (!busyLock.tryReadLock())
-                return;
-
-            try {
-                body();
-            }
-            catch (Throwable e) {
-                U.error(log, "Unhandled exception while processing event.", e);
-            }
-            finally {
-                busyLock.readUnlock();
-            }
-        }
-
-        /**
-         * Handler body.
-         */
-        protected abstract void body() throws Exception;
-    }
-
-    /**
-     *
-     */
-    private class JobLocalState {
-        /** Mappers. */
-        private final Collection<GridHadoopInputSplit> currMappers = new HashSet<>();
-
-        /** Reducers. */
-        private final Collection<Integer> currReducers = new HashSet<>();
-
-        /** Number of completed mappers. */
-        private final AtomicInteger completedMappersCnt = new AtomicInteger();
-
-        /** Cancelled flag. */
-        private boolean cancelled;
-
-        /** Aborted flag. */
-        private boolean aborted;
-
-        /**
-         * @param mapSplit Map split to add.
-         * @return {@code True} if mapper was added.
-         */
-        private boolean addMapper(GridHadoopInputSplit mapSplit) {
-            return currMappers.add(mapSplit);
-        }
-
-        /**
-         * @param rdc Reducer number to add.
-         * @return {@code True} if reducer was added.
-         */
-        private boolean addReducer(int rdc) {
-            return currReducers.add(rdc);
-        }
-
-        /**
-         * Checks whether this split was scheduled for given attempt.
-         *
-         * @param mapSplit Map split to check.
-         * @return {@code True} if mapper was scheduled.
-         */
-        public boolean mapperScheduled(GridHadoopInputSplit mapSplit) {
-            return currMappers.contains(mapSplit);
-        }
-
-        /**
-         * Checks whether this split was scheduled for given attempt.
-         *
-         * @param rdc Reducer number to check.
-         * @return {@code True} if reducer was scheduled.
-         */
-        public boolean reducerScheduled(int rdc) {
-            return currReducers.contains(rdc);
-        }
-
-        /**
-         * @param taskInfo Task info.
-         * @param status Task status.
-         * @param prev Previous closure.
-         */
-        private void onSetupFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
-            final GridHadoopJobId jobId = taskInfo.jobId();
-
-            if (status.state() == FAILED || status.state() == CRASHED)
-                transform(jobId, new CancelJobProcessor(prev, status.failCause()));
-            else
-                transform(jobId, new UpdatePhaseProcessor(prev, PHASE_MAP));
-        }
-
-        /**
-         * @param taskInfo Task info.
-         * @param status Task status.
-         * @param prev Previous closure.
-         */
-        private void onMapFinished(final GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
-            final StackedProcessor prev) {
-            final GridHadoopJobId jobId = taskInfo.jobId();
-
-            boolean lastMapperFinished = completedMappersCnt.incrementAndGet() == currMappers.size();
-
-            if (status.state() == FAILED || status.state() == CRASHED) {
-                // Fail the whole job.
-                transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), status.failCause()));
-
-                return;
-            }
-
-            IgniteInClosure<IgniteInternalFuture<?>> cacheUpdater = new CIX1<IgniteInternalFuture<?>>() {
-                @Override public void applyx(IgniteInternalFuture<?> f) {
-                    Throwable err = null;
-
-                    if (f != null) {
-                        try {
-                            f.get();
-                        }
-                        catch (IgniteCheckedException e) {
-                            err = e;
-                        }
-                    }
-
-                    transform(jobId, new RemoveMappersProcessor(prev, taskInfo.inputSplit(), err));
-                }
-            };
-
-            if (lastMapperFinished)
-                ctx.shuffle().flush(jobId).listenAsync(cacheUpdater);
-            else
-                cacheUpdater.apply(null);
-        }
-
-        /**
-         * @param taskInfo Task info.
-         * @param status Task status.
-         * @param prev Previous closure.
-         */
-        private void onReduceFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status, StackedProcessor prev) {
-            GridHadoopJobId jobId = taskInfo.jobId();
-            if (status.state() == FAILED || status.state() == CRASHED)
-                // Fail the whole job.
-                transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber(), status.failCause()));
-            else
-                transform(jobId, new RemoveReducerProcessor(prev, taskInfo.taskNumber()));
-        }
-
-        /**
-         * @param taskInfo Task info.
-         * @param status Task status.
-         * @param prev Previous closure.
-         */
-        private void onCombineFinished(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status,
-            final StackedProcessor prev) {
-            final GridHadoopJobId jobId = taskInfo.jobId();
-
-            if (status.state() == FAILED || status.state() == CRASHED)
-                // Fail the whole job.
-                transform(jobId, new RemoveMappersProcessor(prev, currMappers, status.failCause()));
-            else {
-                ctx.shuffle().flush(jobId).listenAsync(new CIX1<IgniteInternalFuture<?>>() {
-                    @Override public void applyx(IgniteInternalFuture<?> f) {
-                        Throwable err = null;
-
-                        if (f != null) {
-                            try {
-                                f.get();
-                            }
-                            catch (IgniteCheckedException e) {
-                                err = e;
-                            }
-                        }
-
-                        transform(jobId, new RemoveMappersProcessor(prev, currMappers, err));
-                    }
-                });
-            }
-        }
-
-        /**
-         * @return {@code True} if job was cancelled by this (first) call.
-         */
-        public boolean onCancel() {
-            if (!cancelled && !aborted) {
-                cancelled = true;
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
-         * @return {@code True} if job was aborted this (first) call.
-         */
-        public boolean onAborted() {
-            if (!aborted) {
-                aborted = true;
-
-                return true;
-            }
-
-            return false;
-        }
-    }
-
-    /**
-     * Update job phase transform closure.
-     */
-    private static class UpdatePhaseProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Phase to update. */
-        private final GridHadoopJobPhase phase;
-
-        /**
-         * @param prev Previous closure.
-         * @param phase Phase to update.
-         */
-        private UpdatePhaseProcessor(@Nullable StackedProcessor prev, GridHadoopJobPhase phase) {
-            super(prev);
-
-            this.phase = phase;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            cp.phase(phase);
-        }
-    }
-
-    /**
-     * Remove mapper transform closure.
-     */
-    private static class RemoveMappersProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Mapper split to remove. */
-        private final Collection<GridHadoopInputSplit> splits;
-
-        /** Error. */
-        private final Throwable err;
-
-        /**
-         * @param prev Previous closure.
-         * @param split Mapper split to remove.
-         * @param err Error.
-         */
-        private RemoveMappersProcessor(@Nullable StackedProcessor prev, GridHadoopInputSplit split, Throwable err) {
-            this(prev, Collections.singletonList(split), err);
-        }
-
-        /**
-         * @param prev Previous closure.
-         * @param splits Mapper splits to remove.
-         * @param err Error.
-         */
-        private RemoveMappersProcessor(@Nullable StackedProcessor prev, Collection<GridHadoopInputSplit> splits,
-            Throwable err) {
-            super(prev);
-
-            this.splits = splits;
-            this.err = err;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            Map<GridHadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
-
-            for (GridHadoopInputSplit s : splits)
-                splitsCp.remove(s);
-
-            cp.pendingSplits(splitsCp);
-
-            if (cp.phase() != PHASE_CANCELLING && err != null)
-                cp.failCause(err);
-
-            if (err != null)
-                cp.phase(PHASE_CANCELLING);
-
-            if (splitsCp.isEmpty()) {
-                if (cp.phase() != PHASE_CANCELLING)
-                    cp.phase(PHASE_REDUCE);
-            }
-        }
-    }
-
-    /**
-     * Remove reducer transform closure.
-     */
-    private static class RemoveReducerProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Mapper split to remove. */
-        private final int rdc;
-
-        /** Error. */
-        private Throwable err;
-
-        /**
-         * @param prev Previous closure.
-         * @param rdc Reducer to remove.
-         */
-        private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc) {
-            super(prev);
-
-            this.rdc = rdc;
-        }
-
-        /**
-         * @param prev Previous closure.
-         * @param rdc Reducer to remove.
-         * @param err Error.
-         */
-        private RemoveReducerProcessor(@Nullable StackedProcessor prev, int rdc, Throwable err) {
-            super(prev);
-
-            this.rdc = rdc;
-            this.err = err;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
-
-            rdcCp.remove(rdc);
-
-            cp.pendingReducers(rdcCp);
-
-            if (err != null) {
-                cp.phase(PHASE_CANCELLING);
-                cp.failCause(err);
-            }
-        }
-    }
-
-    /**
-     * Initialize reducers.
-     */
-    private static class InitializeReducersProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Reducers. */
-        private final Collection<Integer> rdc;
-
-        /** Process descriptor for reducers. */
-        private final GridHadoopProcessDescriptor desc;
-
-        /**
-         * @param prev Previous closure.
-         * @param rdc Reducers to initialize.
-         * @param desc External process descriptor.
-         */
-        private InitializeReducersProcessor(@Nullable StackedProcessor prev,
-            Collection<Integer> rdc,
-            GridHadoopProcessDescriptor desc) {
-            super(prev);
-
-            assert !F.isEmpty(rdc);
-            assert desc != null;
-
-            this.rdc = rdc;
-            this.desc = desc;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            Map<Integer, GridHadoopProcessDescriptor> oldMap = meta.reducersAddresses();
-
-            Map<Integer, GridHadoopProcessDescriptor> rdcMap = oldMap == null ?
-                new HashMap<Integer, GridHadoopProcessDescriptor>() : new HashMap<>(oldMap);
-
-            for (Integer r : rdc)
-                rdcMap.put(r, desc);
-
-            cp.reducersAddresses(rdcMap);
-        }
-    }
-
-    /**
-     * Remove reducer transform closure.
-     */
-    private static class CancelJobProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Mapper split to remove. */
-        private final Collection<GridHadoopInputSplit> splits;
-
-        /** Reducers to remove. */
-        private final Collection<Integer> rdc;
-
-        /** Error. */
-        private final Throwable err;
-
-        /**
-         * @param prev Previous closure.
-         * @param err Fail cause.
-         */
-        private CancelJobProcessor(@Nullable StackedProcessor prev, Throwable err) {
-            this(prev, err, null, null);
-        }
-
-        /**
-         * @param prev Previous closure.
-         * @param splits Splits to remove.
-         * @param rdc Reducers to remove.
-         */
-        private CancelJobProcessor(@Nullable StackedProcessor prev,
-            Collection<GridHadoopInputSplit> splits,
-            Collection<Integer> rdc) {
-            this(prev, null, splits, rdc);
-        }
-
-        /**
-         * @param prev Previous closure.
-         * @param err Error.
-         * @param splits Splits to remove.
-         * @param rdc Reducers to remove.
-         */
-        private CancelJobProcessor(@Nullable StackedProcessor prev,
-            Throwable err,
-            Collection<GridHadoopInputSplit> splits,
-            Collection<Integer> rdc) {
-            super(prev);
-
-            this.splits = splits;
-            this.rdc = rdc;
-            this.err = err;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            assert meta.phase() == PHASE_CANCELLING || err != null: "Invalid phase for cancel: " + meta;
-
-            Collection<Integer> rdcCp = new HashSet<>(cp.pendingReducers());
-
-            if (rdc != null)
-                rdcCp.removeAll(rdc);
-
-            cp.pendingReducers(rdcCp);
-
-            Map<GridHadoopInputSplit, Integer> splitsCp = new HashMap<>(cp.pendingSplits());
-
-            if (splits != null) {
-                for (GridHadoopInputSplit s : splits)
-                    splitsCp.remove(s);
-            }
-
-            cp.pendingSplits(splitsCp);
-
-            cp.phase(PHASE_CANCELLING);
-
-            if (err != null)
-                cp.failCause(err);
-        }
-    }
-
-    /**
-     * Increment counter values closure.
-     */
-    private static class IncrementCountersProcessor extends StackedProcessor {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final GridHadoopCounters counters;
-
-        /**
-         * @param prev Previous closure.
-         * @param counters Task counters to add into job counters.
-         */
-        private IncrementCountersProcessor(@Nullable StackedProcessor prev, GridHadoopCounters counters) {
-            super(prev);
-
-            assert counters != null;
-
-            this.counters = counters;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp) {
-            GridHadoopCounters cntrs = new GridHadoopCountersImpl(cp.counters());
-
-            cntrs.merge(counters);
-
-            cp.counters(cntrs);
-        }
-    }
-
-    /**
-     * Abstract stacked closure.
-     */
-    private abstract static class StackedProcessor implements
-        EntryProcessor<GridHadoopJobId, GridHadoopJobMetadata, Void>, Serializable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final StackedProcessor prev;
-
-        /**
-         * @param prev Previous closure.
-         */
-        private StackedProcessor(@Nullable StackedProcessor prev) {
-            this.prev = prev;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Void process(MutableEntry<GridHadoopJobId, GridHadoopJobMetadata> e, Object... args) {
-            GridHadoopJobMetadata val = apply(e.getValue());
-
-            if (val != null)
-                e.setValue(val);
-            else
-                e.remove();;
-
-            return null;
-        }
-
-        /**
-         * @param meta Old value.
-         * @return New value.
-         */
-        private GridHadoopJobMetadata apply(GridHadoopJobMetadata meta) {
-            if (meta == null)
-                return null;
-
-            GridHadoopJobMetadata cp = prev != null ? prev.apply(meta) : new GridHadoopJobMetadata(meta);
-
-            update(meta, cp);
-
-            return cp;
-        }
-
-        /**
-         * Update given job metadata object.
-         *
-         * @param meta Initial job metadata.
-         * @param cp Copy.
-         */
-        protected abstract void update(GridHadoopJobMetadata meta, GridHadoopJobMetadata cp);
-    }
-}


[15/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
# IGNITE-386: WIP on internal namings (2).


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

Branch: refs/heads/ignite-386
Commit: 288709a1b48260e665278037e1beb050ab8ecdb4
Parents: ace354c
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 3 15:55:58 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 3 15:55:58 2015 +0300

----------------------------------------------------------------------
 docs/core-site.ignite.xml                       |    2 +-
 .../ignite/internal/IgniteComponentType.java    |    2 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   92 ++
 .../processors/hadoop/HadoopContext.java        |    2 +-
 .../processors/hadoop/HadoopCounters.java       |   14 +-
 .../internal/processors/hadoop/HadoopImpl.java  |    4 +-
 .../processors/hadoop/HadoopProcessor.java      |  225 ++++
 .../hadoop/IgniteHadoopProcessor.java           |  225 ----
 .../counter/GridHadoopCounterAdapter.java       |  128 ---
 .../hadoop/counter/GridHadoopCountersImpl.java  |  198 ----
 .../counter/GridHadoopFSCounterWriter.java      |   91 --
 .../hadoop/counter/GridHadoopLongCounter.java   |   92 --
 .../counter/GridHadoopPerformanceCounter.java   |  279 -----
 .../hadoop/counter/HadoopCounterAdapter.java    |  128 +++
 .../hadoop/counter/HadoopCountersImpl.java      |  198 ++++
 .../hadoop/counter/HadoopLongCounter.java       |   92 ++
 .../counter/HadoopPerformanceCounter.java       |  279 +++++
 .../fs/GridHadoopDistributedFileSystem.java     |   91 --
 .../hadoop/fs/GridHadoopFileSystemsUtils.java   |   57 -
 .../hadoop/fs/GridHadoopLocalFileSystemV1.java  |   39 -
 .../hadoop/fs/GridHadoopLocalFileSystemV2.java  |   86 --
 .../hadoop/fs/GridHadoopRawLocalFileSystem.java |  304 ------
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 ++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   57 +
 .../hadoop/fs/HadoopLocalFileSystemV1.java      |   39 +
 .../hadoop/fs/HadoopLocalFileSystemV2.java      |   86 ++
 .../hadoop/fs/HadoopRawLocalFileSystem.java     |  304 ++++++
 .../jobtracker/GridHadoopJobMetadata.java       |  305 ------
 .../hadoop/jobtracker/HadoopJobMetadata.java    |  305 ++++++
 .../hadoop/jobtracker/HadoopJobTracker.java     |  104 +-
 .../hadoop/message/GridHadoopMessage.java       |   27 -
 .../hadoop/message/HadoopMessage.java           |   27 +
 .../planner/GridHadoopDefaultMapReducePlan.java |  107 --
 .../GridHadoopDefaultMapReducePlanner.java      |  434 --------
 .../planner/HadoopDefaultMapReducePlan.java     |  107 ++
 .../planner/HadoopDefaultMapReducePlanner.java  |  434 ++++++++
 .../GridHadoopProtocolJobCountersTask.java      |   45 -
 .../proto/GridHadoopProtocolJobStatusTask.java  |   81 --
 .../proto/GridHadoopProtocolKillJobTask.java    |   46 -
 .../proto/GridHadoopProtocolNextTaskIdTask.java |   35 -
 .../proto/GridHadoopProtocolSubmitJobTask.java  |   57 -
 .../proto/GridHadoopProtocolTaskAdapter.java    |  113 --
 .../proto/GridHadoopProtocolTaskArguments.java  |   81 --
 .../hadoop/proto/HadoopClientProtocol.java      |   22 +-
 .../proto/HadoopProtocolJobCountersTask.java    |   45 +
 .../proto/HadoopProtocolJobStatusTask.java      |   81 ++
 .../hadoop/proto/HadoopProtocolKillJobTask.java |   46 +
 .../proto/HadoopProtocolNextTaskIdTask.java     |   35 +
 .../proto/HadoopProtocolSubmitJobTask.java      |   57 +
 .../hadoop/proto/HadoopProtocolTaskAdapter.java |  113 ++
 .../proto/HadoopProtocolTaskArguments.java      |   81 ++
 .../hadoop/shuffle/GridHadoopShuffleAck.java    |   91 --
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |  593 -----------
 .../shuffle/GridHadoopShuffleMessage.java       |  242 -----
 .../hadoop/shuffle/HadoopShuffle.java           |   38 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   91 ++
 .../hadoop/shuffle/HadoopShuffleJob.java        |  593 +++++++++++
 .../hadoop/shuffle/HadoopShuffleMessage.java    |  241 +++++
 .../GridHadoopConcurrentHashMultimap.java       |  611 -----------
 .../collections/GridHadoopHashMultimap.java     |  174 ---
 .../collections/GridHadoopHashMultimapBase.java |  208 ----
 .../shuffle/collections/GridHadoopMultimap.java |  112 --
 .../collections/GridHadoopMultimapBase.java     |  368 -------
 .../shuffle/collections/GridHadoopSkipList.java |  726 -------------
 .../HadoopConcurrentHashMultimap.java           |  611 +++++++++++
 .../shuffle/collections/HadoopHashMultimap.java |  174 +++
 .../collections/HadoopHashMultimapBase.java     |  208 ++++
 .../shuffle/collections/HadoopMultimap.java     |  112 ++
 .../shuffle/collections/HadoopMultimapBase.java |  368 +++++++
 .../shuffle/collections/HadoopSkipList.java     |  726 +++++++++++++
 .../shuffle/streams/GridHadoopDataInStream.java |  170 ---
 .../streams/GridHadoopDataOutStream.java        |  131 ---
 .../streams/GridHadoopOffheapBuffer.java        |  122 ---
 .../shuffle/streams/HadoopDataInStream.java     |  170 +++
 .../shuffle/streams/HadoopDataOutStream.java    |  131 +++
 .../shuffle/streams/HadoopOffheapBuffer.java    |  122 +++
 .../taskexecutor/GridHadoopExecutorService.java |  232 ----
 .../taskexecutor/GridHadoopRunnableTask.java    |   22 +-
 .../taskexecutor/GridHadoopTaskState.java       |   38 -
 .../taskexecutor/GridHadoopTaskStatus.java      |  114 --
 .../HadoopEmbeddedTaskExecutor.java             |    8 +-
 .../taskexecutor/HadoopExecutorService.java     |  231 ++++
 .../taskexecutor/HadoopTaskExecutorAdapter.java |    2 +-
 .../hadoop/taskexecutor/HadoopTaskState.java    |   38 +
 .../hadoop/taskexecutor/HadoopTaskStatus.java   |  114 ++
 .../GridHadoopExternalTaskMetadata.java         |   68 --
 .../GridHadoopJobInfoUpdateRequest.java         |  109 --
 .../GridHadoopPrepareForJobRequest.java         |  126 ---
 .../external/GridHadoopProcessDescriptor.java   |  150 ---
 .../external/GridHadoopProcessStartedAck.java   |   46 -
 .../GridHadoopTaskExecutionRequest.java         |  110 --
 .../external/GridHadoopTaskFinishedMessage.java |   92 --
 .../external/HadoopExternalTaskExecutor.java    |   78 +-
 .../external/HadoopExternalTaskMetadata.java    |   68 ++
 .../external/HadoopJobInfoUpdateRequest.java    |  109 ++
 .../external/HadoopPrepareForJobRequest.java    |  126 +++
 .../external/HadoopProcessDescriptor.java       |  150 +++
 .../external/HadoopProcessStartedAck.java       |   46 +
 .../external/HadoopTaskExecutionRequest.java    |  110 ++
 .../external/HadoopTaskFinishedMessage.java     |   92 ++
 .../child/GridHadoopChildProcessRunner.java     |   72 +-
 .../child/GridHadoopExternalProcessStarter.java |    2 +-
 .../GridHadoopCommunicationClient.java          |    2 +-
 .../GridHadoopExternalCommunication.java        |   52 +-
 .../GridHadoopMarshallerFilter.java             |    2 +-
 .../GridHadoopMessageListener.java              |    4 +-
 .../GridHadoopTcpNioCommunicationClient.java    |    2 +-
 .../hadoop/v1/GridHadoopV1Counter.java          |    4 +-
 .../hadoop/v1/GridHadoopV1Reporter.java         |    2 +-
 .../hadoop/v2/GridHadoopV2Context.java          |    2 +-
 .../hadoop/v2/GridHadoopV2Counter.java          |    4 +-
 .../processors/hadoop/v2/GridHadoopV2Job.java   |    2 +-
 .../v2/GridHadoopV2JobResourceManager.java      |    2 +-
 .../hadoop/v2/GridHadoopV2TaskContext.java      |    4 +-
 .../hadoop/GridHadoopAbstractSelfTest.java      |    2 +-
 .../hadoop/GridHadoopCommandLineTest.java       |    4 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java | 1005 ------------------
 .../hadoop/GridHadoopFileSystemsTest.java       |    6 +-
 .../hadoop/GridHadoopMapReduceTest.java         |    7 +-
 .../GridHadoopTestRoundRobinMrPlanner.java      |    2 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 1005 ++++++++++++++++++
 ...ridHadoopConcurrentHashMultimapSelftest.java |   12 +-
 .../collections/GridHadoopHashMapSelfTest.java  |    6 +-
 .../collections/GridHadoopSkipListSelfTest.java |   14 +-
 .../streams/GridHadoopDataStreamSelfTest.java   |    4 +-
 .../GridHadoopExecutorServiceTest.java          |  119 ---
 .../taskexecutor/HadoopExecutorServiceTest.java |  119 +++
 ...GridHadoopExternalCommunicationSelfTest.java |    6 +-
 .../testsuites/IgniteHadoopTestSuite.java       |    2 +-
 129 files changed, 8937 insertions(+), 8937 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/docs/core-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/core-site.ignite.xml b/docs/core-site.ignite.xml
index 1146576..8b8e634 100644
--- a/docs/core-site.ignite.xml
+++ b/docs/core-site.ignite.xml
@@ -73,7 +73,7 @@
     <!--
     <property>
         <name>ignite.counters.writer</name>
-        <value>org.apache.ignite.internal.processors.hadoop.counter.GridHadoopFSCounterWriter</value>
+        <value>org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter</value>
     </property>
     -->
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
index d0e487a..a51800e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteComponentType.java
@@ -36,7 +36,7 @@ public enum IgniteComponentType {
     /** Hadoop. */
     HADOOP(
         "org.apache.ignite.internal.processors.hadoop.IgniteHadoopNoopProcessor",
-        "org.apache.ignite.internal.processors.hadoop.IgniteHadoopProcessor",
+        "org.apache.ignite.internal.processors.hadoop.HadoopProcessor",
         "ignite-hadoop"
     ),
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
new file mode 100644
index 0000000..449cff2
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
@@ -0,0 +1,92 @@
+/*
+ * 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.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.*;
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.util.typedef.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Statistic writer implementation that writes info into any Hadoop file system.
+ */
+public class IgniteHadoopFileSystemCounterWriter implements GridHadoopCounterWriter {
+    /** */
+    public static final String PERFORMANCE_COUNTER_FILE_NAME = "performance";
+
+    /** */
+    private static final String DEFAULT_USER_NAME = "anonymous";
+
+    /** */
+    public static final String COUNTER_WRITER_DIR_PROPERTY = "ignite.counters.fswriter.directory";
+
+    /** */
+    private static final String USER_MACRO = "${USER}";
+
+    /** */
+    private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO;
+
+    /** {@inheritDoc} */
+    @Override public void write(GridHadoopJobInfo jobInfo, GridHadoopJobId jobId, GridHadoopCounters cntrs)
+        throws IgniteCheckedException {
+
+        Configuration hadoopCfg = new Configuration();
+
+        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
+            hadoopCfg.set(e.getKey(), e.getValue());
+
+        String user = jobInfo.user();
+
+        if (F.isEmpty(user))
+            user = DEFAULT_USER_NAME;
+
+        String dir = jobInfo.property(COUNTER_WRITER_DIR_PROPERTY);
+
+        if (dir == null)
+            dir = DEFAULT_COUNTER_WRITER_DIR;
+
+        Path jobStatPath = new Path(new Path(dir.replace(USER_MACRO, user)), jobId.toString());
+
+        HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null);
+
+        try {
+            FileSystem fs = jobStatPath.getFileSystem(hadoopCfg);
+
+            fs.mkdirs(jobStatPath);
+
+            try (PrintStream out = new PrintStream(fs.create(new Path(jobStatPath, PERFORMANCE_COUNTER_FILE_NAME)))) {
+                for (T2<String, Long> evt : perfCntr.evts()) {
+                    out.print(evt.get1());
+                    out.print(':');
+                    out.println(evt.get2().toString());
+                }
+
+                out.flush();
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
index bb707c8..d897b6c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
@@ -143,7 +143,7 @@ public class HadoopContext {
      * @param meta Job metadata.
      * @return {@code true} If local node is participating in job execution.
      */
-    public boolean isParticipating(GridHadoopJobMetadata meta) {
+    public boolean isParticipating(HadoopJobMetadata meta) {
         UUID locNodeId = localNodeId();
 
         if (locNodeId.equals(meta.submitNodeId()))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
index c7f0157..ad699ec 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
@@ -31,7 +31,7 @@ import java.util.*;
  */
 public class HadoopCounters extends Counters {
     /** */
-    private final Map<T2<String,String>,GridHadoopLongCounter> cntrs = new HashMap<>();
+    private final Map<T2<String,String>,HadoopLongCounter> cntrs = new HashMap<>();
 
     /**
      * Creates new instance based on given counters.
@@ -40,8 +40,8 @@ public class HadoopCounters extends Counters {
      */
     public HadoopCounters(GridHadoopCounters cntrs) {
         for (GridHadoopCounter cntr : cntrs.all())
-            if (cntr instanceof GridHadoopLongCounter)
-                this.cntrs.put(new T2<>(cntr.group(), cntr.name()), (GridHadoopLongCounter) cntr);
+            if (cntr instanceof HadoopLongCounter)
+                this.cntrs.put(new T2<>(cntr.group(), cntr.name()), (HadoopLongCounter) cntr);
     }
 
     /** {@inheritDoc} */
@@ -184,7 +184,7 @@ public class HadoopCounters extends Counters {
     public Iterator<Counter> iterateGroup(String grpName) {
         Collection<Counter> grpCounters = new ArrayList<>();
 
-        for (GridHadoopLongCounter counter : cntrs.values()) {
+        for (HadoopLongCounter counter : cntrs.values()) {
             if (grpName.equals(counter.group()))
                 grpCounters.add(new GridHadoopV2Counter(counter));
         }
@@ -203,12 +203,12 @@ public class HadoopCounters extends Counters {
     public Counter findCounter(String grpName, String cntrName, boolean create) {
         T2<String, String> key = new T2<>(grpName, cntrName);
 
-        GridHadoopLongCounter internalCntr = cntrs.get(key);
+        HadoopLongCounter internalCntr = cntrs.get(key);
 
         if (internalCntr == null & create) {
-            internalCntr = new GridHadoopLongCounter(grpName,cntrName);
+            internalCntr = new HadoopLongCounter(grpName,cntrName);
 
-            cntrs.put(key, new GridHadoopLongCounter(grpName,cntrName));
+            cntrs.put(key, new HadoopLongCounter(grpName,cntrName));
         }
 
         return internalCntr == null ? null : new GridHadoopV2Counter(internalCntr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
index 80fd995..b87e7f8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopImpl.java
@@ -27,7 +27,7 @@ import org.jetbrains.annotations.*;
  */
 public class HadoopImpl implements GridHadoop {
     /** Hadoop processor. */
-    private final IgniteHadoopProcessor proc;
+    private final HadoopProcessor proc;
 
     /** Busy lock. */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
@@ -37,7 +37,7 @@ public class HadoopImpl implements GridHadoop {
      *
      * @param proc Hadoop processor.
      */
-    HadoopImpl(IgniteHadoopProcessor proc) {
+    HadoopImpl(HadoopProcessor proc) {
         this.proc = proc;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
new file mode 100644
index 0000000..1f50b0c
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
@@ -0,0 +1,225 @@
+/*
+ * 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.jobtracker.*;
+import org.apache.ignite.internal.processors.hadoop.planner.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopClassLoader.*;
+
+/**
+ * Hadoop processor.
+ */
+public class HadoopProcessor extends IgniteHadoopProcessorAdapter {
+    /** Job ID counter. */
+    private final AtomicInteger idCtr = new AtomicInteger();
+
+    /** Hadoop context. */
+    @GridToStringExclude
+    private HadoopContext hctx;
+
+    /** Hadoop facade for public API. */
+    @GridToStringExclude
+    private GridHadoop hadoop;
+
+    /**
+     * @param ctx Kernal context.
+     */
+    public HadoopProcessor(GridKernalContext ctx) {
+        super(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteCheckedException {
+        if (ctx.isDaemon())
+            return;
+
+        GridHadoopConfiguration cfg = ctx.config().getHadoopConfiguration();
+
+        if (cfg == null)
+            cfg = new GridHadoopConfiguration();
+        else
+            cfg = new GridHadoopConfiguration(cfg);
+
+        initializeDefaults(cfg);
+
+        validate(cfg);
+
+        if (hadoopHome() != null)
+            U.quietAndInfo(log, "HADOOP_HOME is set to " + hadoopHome());
+
+        boolean ok = false;
+
+        try { // Check for Hadoop installation.
+            hadoopUrls();
+
+            ok = true;
+        }
+        catch (IgniteCheckedException e) {
+            U.quietAndWarn(log, e.getMessage());
+        }
+
+        if (ok) {
+            hctx = new HadoopContext(
+                ctx,
+                cfg,
+                new HadoopJobTracker(),
+                cfg.isExternalExecution() ? new HadoopExternalTaskExecutor() : new HadoopEmbeddedTaskExecutor(),
+                new HadoopShuffle());
+
+
+            for (HadoopComponent c : hctx.components())
+                c.start(hctx);
+
+            hadoop = new HadoopImpl(this);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopProcessor.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        super.stop(cancel);
+
+        if (hctx == null)
+            return;
+
+        List<HadoopComponent> components = hctx.components();
+
+        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
+            HadoopComponent c = it.previous();
+
+            c.stop(cancel);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStart() throws IgniteCheckedException {
+        super.onKernalStart();
+
+        if (hctx == null)
+            return;
+
+        for (HadoopComponent c : hctx.components())
+            c.onKernalStart();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        super.onKernalStop(cancel);
+
+        if (hctx == null)
+            return;
+
+        List<HadoopComponent> components = hctx.components();
+
+        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
+            HadoopComponent c = it.previous();
+
+            c.onKernalStop(cancel);
+        }
+    }
+
+    /**
+     * Gets Hadoop context.
+     *
+     * @return Hadoop context.
+     */
+    public HadoopContext context() {
+        return hctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoop hadoop() {
+        if (hadoop == null)
+            throw new IllegalStateException("Hadoop accelerator is disabled (Hadoop is not in classpath, " +
+                "is HADOOP_HOME environment variable set?)");
+
+        return hadoop;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopConfiguration config() {
+        return hctx.configuration();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobId nextJobId() {
+        return new GridHadoopJobId(ctx.localNodeId(), idCtr.incrementAndGet());
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+        return hctx.jobTracker().submit(jobId, jobInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
+        return hctx.jobTracker().status(jobId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException {
+        return hctx.jobTracker().jobCounters(jobId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+        return hctx.jobTracker().finishFuture(jobId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException {
+        return hctx.jobTracker().killJob(jobId);
+    }
+
+    /**
+     * Initializes default hadoop configuration.
+     *
+     * @param cfg Hadoop configuration.
+     */
+    private void initializeDefaults(GridHadoopConfiguration cfg) {
+        if (cfg.getMapReducePlanner() == null)
+            cfg.setMapReducePlanner(new HadoopDefaultMapReducePlanner());
+    }
+
+    /**
+     * Validates Grid and Hadoop configuration for correctness.
+     *
+     * @param hadoopCfg Hadoop configuration.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void validate(GridHadoopConfiguration hadoopCfg) throws IgniteCheckedException {
+        if (ctx.config().isPeerClassLoadingEnabled())
+            throw new IgniteCheckedException("Peer class loading cannot be used with Hadoop (disable it using " +
+                "GridConfiguration.setPeerClassLoadingEnabled()).");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
deleted file mode 100644
index 63e4854..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
+++ /dev/null
@@ -1,225 +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.jobtracker.*;
-import org.apache.ignite.internal.processors.hadoop.planner.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.hadoop.HadoopClassLoader.*;
-
-/**
- * Hadoop processor.
- */
-public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
-    /** Job ID counter. */
-    private final AtomicInteger idCtr = new AtomicInteger();
-
-    /** Hadoop context. */
-    @GridToStringExclude
-    private HadoopContext hctx;
-
-    /** Hadoop facade for public API. */
-    @GridToStringExclude
-    private GridHadoop hadoop;
-
-    /**
-     * @param ctx Kernal context.
-     */
-    public IgniteHadoopProcessor(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void start() throws IgniteCheckedException {
-        if (ctx.isDaemon())
-            return;
-
-        GridHadoopConfiguration cfg = ctx.config().getHadoopConfiguration();
-
-        if (cfg == null)
-            cfg = new GridHadoopConfiguration();
-        else
-            cfg = new GridHadoopConfiguration(cfg);
-
-        initializeDefaults(cfg);
-
-        validate(cfg);
-
-        if (hadoopHome() != null)
-            U.quietAndInfo(log, "HADOOP_HOME is set to " + hadoopHome());
-
-        boolean ok = false;
-
-        try { // Check for Hadoop installation.
-            hadoopUrls();
-
-            ok = true;
-        }
-        catch (IgniteCheckedException e) {
-            U.quietAndWarn(log, e.getMessage());
-        }
-
-        if (ok) {
-            hctx = new HadoopContext(
-                ctx,
-                cfg,
-                new HadoopJobTracker(),
-                cfg.isExternalExecution() ? new HadoopExternalTaskExecutor() : new HadoopEmbeddedTaskExecutor(),
-                new HadoopShuffle());
-
-
-            for (HadoopComponent c : hctx.components())
-                c.start(hctx);
-
-            hadoop = new HadoopImpl(this);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(IgniteHadoopProcessor.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        super.stop(cancel);
-
-        if (hctx == null)
-            return;
-
-        List<HadoopComponent> components = hctx.components();
-
-        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
-            HadoopComponent c = it.previous();
-
-            c.stop(cancel);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStart() throws IgniteCheckedException {
-        super.onKernalStart();
-
-        if (hctx == null)
-            return;
-
-        for (HadoopComponent c : hctx.components())
-            c.onKernalStart();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStop(boolean cancel) {
-        super.onKernalStop(cancel);
-
-        if (hctx == null)
-            return;
-
-        List<HadoopComponent> components = hctx.components();
-
-        for (ListIterator<HadoopComponent> it = components.listIterator(components.size()); it.hasPrevious();) {
-            HadoopComponent c = it.previous();
-
-            c.onKernalStop(cancel);
-        }
-    }
-
-    /**
-     * Gets Hadoop context.
-     *
-     * @return Hadoop context.
-     */
-    public HadoopContext context() {
-        return hctx;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoop hadoop() {
-        if (hadoop == null)
-            throw new IllegalStateException("Hadoop accelerator is disabled (Hadoop is not in classpath, " +
-                "is HADOOP_HOME environment variable set?)");
-
-        return hadoop;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopConfiguration config() {
-        return hctx.configuration();
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobId nextJobId() {
-        return new GridHadoopJobId(ctx.localNodeId(), idCtr.incrementAndGet());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
-        return hctx.jobTracker().submit(jobId, jobInfo);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobStatus status(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return hctx.jobTracker().status(jobId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopCounters counters(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return hctx.jobTracker().jobCounters(jobId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return hctx.jobTracker().finishFuture(jobId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean kill(GridHadoopJobId jobId) throws IgniteCheckedException {
-        return hctx.jobTracker().killJob(jobId);
-    }
-
-    /**
-     * Initializes default hadoop configuration.
-     *
-     * @param cfg Hadoop configuration.
-     */
-    private void initializeDefaults(GridHadoopConfiguration cfg) {
-        if (cfg.getMapReducePlanner() == null)
-            cfg.setMapReducePlanner(new GridHadoopDefaultMapReducePlanner());
-    }
-
-    /**
-     * Validates Grid and Hadoop configuration for correctness.
-     *
-     * @param hadoopCfg Hadoop configuration.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void validate(GridHadoopConfiguration hadoopCfg) throws IgniteCheckedException {
-        if (ctx.config().isPeerClassLoadingEnabled())
-            throw new IgniteCheckedException("Peer class loading cannot be used with Hadoop (disable it using " +
-                "GridConfiguration.setPeerClassLoadingEnabled()).");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCounterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCounterAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCounterAdapter.java
deleted file mode 100644
index 9e46846..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCounterAdapter.java
+++ /dev/null
@@ -1,128 +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.counter;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Default Hadoop counter implementation.
- */
-public abstract class GridHadoopCounterAdapter implements GridHadoopCounter, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Counter group name. */
-    private String grp;
-
-    /** Counter name. */
-    private String name;
-
-    /**
-     * Default constructor required by {@link Externalizable}.
-     */
-    protected GridHadoopCounterAdapter() {
-        // No-op.
-    }
-
-    /**
-     * Creates new counter with given group and name.
-     *
-     * @param grp Counter group name.
-     * @param name Counter name.
-     */
-    protected GridHadoopCounterAdapter(String grp, String name) {
-        assert grp != null : "counter must have group";
-        assert name != null : "counter must have name";
-
-        this.grp = grp;
-        this.name = name;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return name;
-    }
-
-    /** {@inheritDoc} */
-    @Override @Nullable public String group() {
-        return grp;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeUTF(grp);
-        out.writeUTF(name);
-        writeValue(out);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        grp = in.readUTF();
-        name = in.readUTF();
-        readValue(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GridHadoopCounterAdapter cntr = (GridHadoopCounterAdapter)o;
-
-        if (!grp.equals(cntr.grp))
-            return false;
-        if (!name.equals(cntr.name))
-            return false;
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = grp.hashCode();
-        res = 31 * res + name.hashCode();
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopCounterAdapter.class, this);
-    }
-
-    /**
-     * Writes value of this counter to output.
-     *
-     * @param out Output.
-     * @throws IOException If failed.
-     */
-    protected abstract void writeValue(ObjectOutput out) throws IOException;
-
-    /**
-     * Read value of this counter from input.
-     *
-     * @param in Input.
-     * @throws IOException If failed.
-     */
-    protected abstract void readValue(ObjectInput in) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCountersImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCountersImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCountersImpl.java
deleted file mode 100644
index 92d54af..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopCountersImpl.java
+++ /dev/null
@@ -1,198 +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.counter;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jdk8.backport.*;
-
-import java.io.*;
-import java.lang.reflect.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Default in-memory counters store.
- */
-public class GridHadoopCountersImpl implements GridHadoopCounters, Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private final ConcurrentMap<CounterKey, GridHadoopCounter> cntrsMap = new ConcurrentHashMap8<>();
-
-    /**
-     * Default constructor. Creates new instance without counters.
-     */
-    public GridHadoopCountersImpl() {
-        // No-op.
-    }
-
-    /**
-     * Creates new instance that contain given counters.
-     *
-     * @param cntrs Counters to store.
-     */
-    public GridHadoopCountersImpl(Iterable<GridHadoopCounter> cntrs) {
-        addCounters(cntrs, true);
-    }
-
-    /**
-     * Copy constructor.
-     *
-     * @param cntrs Counters to copy.
-     */
-    public GridHadoopCountersImpl(GridHadoopCounters cntrs) {
-        this(cntrs.all());
-    }
-
-    /**
-     * Creates counter instance.
-     *
-     * @param cls Class of the counter.
-     * @param grp Group name.
-     * @param name Counter name.
-     * @return Counter.
-     */
-    private <T extends GridHadoopCounter> T createCounter(Class<? extends GridHadoopCounter> cls, String grp,
-        String name) {
-        try {
-            Constructor constructor = cls.getConstructor(String.class, String.class);
-
-            return (T)constructor.newInstance(grp, name);
-        }
-        catch (Exception e) {
-            throw new IgniteException(e);
-        }
-    }
-
-    /**
-     * Adds counters collection in addition to existing counters.
-     *
-     * @param cntrs Counters to add.
-     * @param cp Whether to copy counters or not.
-     */
-    private void addCounters(Iterable<GridHadoopCounter> cntrs, boolean cp) {
-        assert cntrs != null;
-
-        for (GridHadoopCounter cntr : cntrs) {
-            if (cp) {
-                GridHadoopCounter cntrCp = createCounter(cntr.getClass(), cntr.group(), cntr.name());
-
-                cntrCp.merge(cntr);
-
-                cntr = cntrCp;
-            }
-
-            cntrsMap.put(new CounterKey(cntr.getClass(), cntr.group(), cntr.name()), cntr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T extends GridHadoopCounter> T counter(String grp, String name, Class<T> cls) {
-        assert cls != null;
-
-        CounterKey mapKey = new CounterKey(cls, grp, name);
-
-        T cntr = (T)cntrsMap.get(mapKey);
-
-        if (cntr == null) {
-            cntr = createCounter(cls, grp, name);
-
-            T old = (T)cntrsMap.putIfAbsent(mapKey, cntr);
-
-            if (old != null)
-                return old;
-        }
-
-        return cntr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<GridHadoopCounter> all() {
-        return cntrsMap.values();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void merge(GridHadoopCounters other) {
-        for (GridHadoopCounter counter : other.all())
-            counter(counter.group(), counter.name(), counter.getClass()).merge(counter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeCollection(out, cntrsMap.values());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        addCounters(U.<GridHadoopCounter>readCollection(in), false);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GridHadoopCountersImpl counters = (GridHadoopCountersImpl)o;
-
-        return cntrsMap.equals(counters.cntrsMap);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return cntrsMap.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopCountersImpl.class, this, "counters", cntrsMap.values());
-    }
-
-    /**
-     * The tuple of counter identifier components for more readable code.
-     */
-    private static class CounterKey extends GridTuple3<Class<? extends GridHadoopCounter>, String, String> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /**
-         * Constructor.
-         *
-         * @param cls Class of the counter.
-         * @param grp Group name.
-         * @param name Counter name.
-         */
-        private CounterKey(Class<? extends GridHadoopCounter> cls, String grp, String name) {
-            super(cls, grp, name);
-        }
-
-        /**
-         * Empty constructor required by {@link Externalizable}.
-         */
-        public CounterKey() {
-            // No-op.
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
deleted file mode 100644
index d603d76..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopFSCounterWriter.java
+++ /dev/null
@@ -1,91 +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.counter;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Statistic writer implementation that writes info into any Hadoop file system.
- */
-public class GridHadoopFSCounterWriter implements GridHadoopCounterWriter {
-    /** */
-    public static final String PERFORMANCE_COUNTER_FILE_NAME = "performance";
-
-    /** */
-    private static final String DEFAULT_USER_NAME = "anonymous";
-
-    /** */
-    public static final String COUNTER_WRITER_DIR_PROPERTY = "ignite.counters.fswriter.directory";
-
-    /** */
-    private static final String USER_MACRO = "${USER}";
-
-    /** */
-    private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO;
-
-    /** {@inheritDoc} */
-    @Override public void write(GridHadoopJobInfo jobInfo, GridHadoopJobId jobId, GridHadoopCounters cntrs)
-        throws IgniteCheckedException {
-
-        Configuration hadoopCfg = new Configuration();
-
-        for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
-            hadoopCfg.set(e.getKey(), e.getValue());
-
-        String user = jobInfo.user();
-
-        if (F.isEmpty(user))
-            user = DEFAULT_USER_NAME;
-
-        String dir = jobInfo.property(COUNTER_WRITER_DIR_PROPERTY);
-
-        if (dir == null)
-            dir = DEFAULT_COUNTER_WRITER_DIR;
-
-        Path jobStatPath = new Path(new Path(dir.replace(USER_MACRO, user)), jobId.toString());
-
-        GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(cntrs, null);
-
-        try {
-            FileSystem fs = jobStatPath.getFileSystem(hadoopCfg);
-
-            fs.mkdirs(jobStatPath);
-
-            try (PrintStream out = new PrintStream(fs.create(new Path(jobStatPath, PERFORMANCE_COUNTER_FILE_NAME)))) {
-                for (T2<String, Long> evt : perfCntr.evts()) {
-                    out.print(evt.get1());
-                    out.print(':');
-                    out.println(evt.get2().toString());
-                }
-
-                out.flush();
-            }
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopLongCounter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopLongCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopLongCounter.java
deleted file mode 100644
index 67af49f..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopLongCounter.java
+++ /dev/null
@@ -1,92 +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.counter;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-
-import java.io.*;
-
-/**
- * Standard hadoop counter to use via original Hadoop API in Hadoop jobs.
- */
-public class GridHadoopLongCounter extends GridHadoopCounterAdapter {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** The counter value. */
-    private long val;
-
-    /**
-     * Default constructor required by {@link Externalizable}.
-     */
-    public GridHadoopLongCounter() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param grp Group name.
-     * @param name Counter name.
-     */
-    public GridHadoopLongCounter(String grp, String name) {
-        super(grp, name);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeValue(ObjectOutput out) throws IOException {
-        out.writeLong(val);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void readValue(ObjectInput in) throws IOException {
-        val = in.readLong();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void merge(GridHadoopCounter cntr) {
-        val += ((GridHadoopLongCounter)cntr).val;
-    }
-
-    /**
-     * Gets current value of this counter.
-     *
-     * @return Current value.
-     */
-    public long value() {
-        return val;
-    }
-
-    /**
-     * Sets current value by the given value.
-     *
-     * @param val Value to set.
-     */
-    public void value(long val) {
-        this.val = val;
-    }
-
-    /**
-     * Increment this counter by the given value.
-     *
-     * @param i Value to increase this counter by.
-     */
-    public void increment(long i) {
-        val += i;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
deleted file mode 100644
index 263a075..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/GridHadoopPerformanceCounter.java
+++ /dev/null
@@ -1,279 +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.counter;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
-
-/**
- * Counter for the job statistics accumulation.
- */
-public class GridHadoopPerformanceCounter extends GridHadoopCounterAdapter {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** The group name for this counter. */
-    private static final String GROUP_NAME = "SYSTEM";
-
-    /** The counter name for this counter. */
-    private static final String COUNTER_NAME = "PERFORMANCE";
-
-    /** Events collections. */
-    private Collection<T2<String,Long>> evts = new ArrayList<>();
-
-    /** Node id to insert into the event info. */
-    private UUID nodeId;
-
-    /** */
-    private int reducerNum;
-
-    /** */
-    private volatile Long firstShuffleMsg;
-
-    /** */
-    private volatile Long lastShuffleMsg;
-
-    /**
-     * Default constructor required by {@link Externalizable}.
-     */
-    public GridHadoopPerformanceCounter() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param grp Group name.
-     * @param name Counter name.
-     */
-    public GridHadoopPerformanceCounter(String grp, String name) {
-        super(grp, name);
-    }
-
-    /**
-     * Constructor to create instance to use this as helper.
-     *
-     * @param nodeId Id of the work node.
-     */
-    public GridHadoopPerformanceCounter(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeValue(ObjectOutput out) throws IOException {
-        U.writeCollection(out, evts);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void readValue(ObjectInput in) throws IOException {
-        try {
-            evts = U.readCollection(in);
-        }
-        catch (ClassNotFoundException e) {
-            throw new IOException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void merge(GridHadoopCounter cntr) {
-        evts.addAll(((GridHadoopPerformanceCounter)cntr).evts);
-    }
-
-    /**
-     * Gets the events collection.
-     *
-     * @return Collection of event.
-     */
-    public Collection<T2<String, Long>> evts() {
-        return evts;
-    }
-
-    /**
-     * Generate name that consists of some event information.
-     *
-     * @param info Task info.
-     * @param evtType The type of the event.
-     * @return String contains necessary event information.
-     */
-    private String eventName(GridHadoopTaskInfo info, String evtType) {
-        return eventName(info.type().toString(), info.taskNumber(), evtType);
-    }
-
-    /**
-     * Generate name that consists of some event information.
-     *
-     * @param taskType Task type.
-     * @param taskNum Number of the task.
-     * @param evtType The type of the event.
-     * @return String contains necessary event information.
-     */
-    private String eventName(String taskType, int taskNum, String evtType) {
-        assert nodeId != null;
-
-        return taskType + " " + taskNum + " " + evtType + " " + nodeId;
-    }
-
-    /**
-     * Adds event of the task submission (task instance creation).
-     *
-     * @param info Task info.
-     * @param ts Timestamp of the event.
-     */
-    public void onTaskSubmit(GridHadoopTaskInfo info, long ts) {
-        evts.add(new T2<>(eventName(info, "submit"), ts));
-    }
-
-    /**
-     * Adds event of the task preparation.
-     *
-     * @param info Task info.
-     * @param ts Timestamp of the event.
-     */
-    public void onTaskPrepare(GridHadoopTaskInfo info, long ts) {
-        evts.add(new T2<>(eventName(info, "prepare"), ts));
-    }
-
-    /**
-     * Adds event of the task finish.
-     *
-     * @param info Task info.
-     * @param ts Timestamp of the event.
-     */
-    public void onTaskFinish(GridHadoopTaskInfo info, long ts) {
-        if (info.type() == GridHadoopTaskType.REDUCE && lastShuffleMsg != null) {
-            evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "start"), firstShuffleMsg));
-            evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "finish"), lastShuffleMsg));
-
-            lastShuffleMsg = null;
-        }
-
-        evts.add(new T2<>(eventName(info, "finish"), ts));
-    }
-
-    /**
-     * Adds event of the task run.
-     *
-     * @param info Task info.
-     * @param ts Timestamp of the event.
-     */
-    public void onTaskStart(GridHadoopTaskInfo info, long ts) {
-        evts.add(new T2<>(eventName(info, "start"), ts));
-    }
-
-    /**
-     * Adds event of the job preparation.
-     *
-     * @param ts Timestamp of the event.
-     */
-    public void onJobPrepare(long ts) {
-        assert nodeId != null;
-
-        evts.add(new T2<>("JOB prepare " + nodeId, ts));
-    }
-
-    /**
-     * Adds event of the job start.
-     *
-     * @param ts Timestamp of the event.
-     */
-    public void onJobStart(long ts) {
-        assert nodeId != null;
-
-        evts.add(new T2<>("JOB start " + nodeId, ts));
-    }
-
-    /**
-     * Adds client submission events from job info.
-     *
-     * @param info Job info.
-     */
-    public void clientSubmissionEvents(GridHadoopJobInfo info) {
-        assert nodeId != null;
-
-        addEventFromProperty("JOB requestId", info, REQ_NEW_JOBID_TS_PROPERTY);
-        addEventFromProperty("JOB responseId", info, RESPONSE_NEW_JOBID_TS_PROPERTY);
-        addEventFromProperty("JOB submit", info, JOB_SUBMISSION_START_TS_PROPERTY);
-    }
-
-    /**
-     * Adds event with timestamp from some property in job info.
-     *
-     * @param evt Event type and phase.
-     * @param info Job info.
-     * @param propName Property name to get timestamp.
-     */
-    private void addEventFromProperty(String evt, GridHadoopJobInfo info, String propName) {
-        String val = info.property(propName);
-
-        if (!F.isEmpty(val)) {
-            try {
-                evts.add(new T2<>(evt + " " + nodeId, Long.parseLong(val)));
-            }
-            catch (NumberFormatException e) {
-                throw new IllegalStateException("Invalid value '" + val + "' of property '" + propName + "'", e);
-            }
-        }
-    }
-
-    /**
-     * Registers shuffle message event.
-     *
-     * @param reducerNum Number of reducer that receives the data.
-     * @param ts Timestamp of the event.
-     */
-    public void onShuffleMessage(int reducerNum, long ts) {
-        this.reducerNum = reducerNum;
-
-        if (firstShuffleMsg == null)
-            firstShuffleMsg = ts;
-
-        lastShuffleMsg = ts;
-    }
-
-    /**
-     * Gets system predefined performance counter from the GridHadoopCounters object.
-     *
-     * @param cntrs GridHadoopCounters object.
-     * @param nodeId Node id for methods that adds events. It may be null if you don't use ones.
-     * @return Predefined performance counter.
-     */
-    public static GridHadoopPerformanceCounter getCounter(GridHadoopCounters cntrs, @Nullable UUID nodeId) {
-        GridHadoopPerformanceCounter cntr = cntrs.counter(GROUP_NAME, COUNTER_NAME, GridHadoopPerformanceCounter.class);
-
-        if (nodeId != null)
-            cntr.nodeId(nodeId);
-
-        return cntrs.counter(GROUP_NAME, COUNTER_NAME, GridHadoopPerformanceCounter.class);
-    }
-
-    /**
-     * Sets the nodeId field.
-     *
-     * @param nodeId Node id.
-     */
-    private void nodeId(UUID nodeId) {
-        this.nodeId = nodeId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java
new file mode 100644
index 0000000..3fdce14
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterAdapter.java
@@ -0,0 +1,128 @@
+/*
+ * 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.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Default Hadoop counter implementation.
+ */
+public abstract class HadoopCounterAdapter implements GridHadoopCounter, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Counter group name. */
+    private String grp;
+
+    /** Counter name. */
+    private String name;
+
+    /**
+     * Default constructor required by {@link Externalizable}.
+     */
+    protected HadoopCounterAdapter() {
+        // No-op.
+    }
+
+    /**
+     * Creates new counter with given group and name.
+     *
+     * @param grp Counter group name.
+     * @param name Counter name.
+     */
+    protected HadoopCounterAdapter(String grp, String name) {
+        assert grp != null : "counter must have group";
+        assert name != null : "counter must have name";
+
+        this.grp = grp;
+        this.name = name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return name;
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public String group() {
+        return grp;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeUTF(grp);
+        out.writeUTF(name);
+        writeValue(out);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        grp = in.readUTF();
+        name = in.readUTF();
+        readValue(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        HadoopCounterAdapter cntr = (HadoopCounterAdapter)o;
+
+        if (!grp.equals(cntr.grp))
+            return false;
+        if (!name.equals(cntr.name))
+            return false;
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = grp.hashCode();
+        res = 31 * res + name.hashCode();
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopCounterAdapter.class, this);
+    }
+
+    /**
+     * Writes value of this counter to output.
+     *
+     * @param out Output.
+     * @throws IOException If failed.
+     */
+    protected abstract void writeValue(ObjectOutput out) throws IOException;
+
+    /**
+     * Read value of this counter from input.
+     *
+     * @param in Input.
+     * @throws IOException If failed.
+     */
+    protected abstract void readValue(ObjectInput in) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCountersImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCountersImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCountersImpl.java
new file mode 100644
index 0000000..01b1473
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCountersImpl.java
@@ -0,0 +1,198 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jdk8.backport.*;
+
+import java.io.*;
+import java.lang.reflect.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Default in-memory counters store.
+ */
+public class HadoopCountersImpl implements GridHadoopCounters, Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final ConcurrentMap<CounterKey, GridHadoopCounter> cntrsMap = new ConcurrentHashMap8<>();
+
+    /**
+     * Default constructor. Creates new instance without counters.
+     */
+    public HadoopCountersImpl() {
+        // No-op.
+    }
+
+    /**
+     * Creates new instance that contain given counters.
+     *
+     * @param cntrs Counters to store.
+     */
+    public HadoopCountersImpl(Iterable<GridHadoopCounter> cntrs) {
+        addCounters(cntrs, true);
+    }
+
+    /**
+     * Copy constructor.
+     *
+     * @param cntrs Counters to copy.
+     */
+    public HadoopCountersImpl(GridHadoopCounters cntrs) {
+        this(cntrs.all());
+    }
+
+    /**
+     * Creates counter instance.
+     *
+     * @param cls Class of the counter.
+     * @param grp Group name.
+     * @param name Counter name.
+     * @return Counter.
+     */
+    private <T extends GridHadoopCounter> T createCounter(Class<? extends GridHadoopCounter> cls, String grp,
+        String name) {
+        try {
+            Constructor constructor = cls.getConstructor(String.class, String.class);
+
+            return (T)constructor.newInstance(grp, name);
+        }
+        catch (Exception e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Adds counters collection in addition to existing counters.
+     *
+     * @param cntrs Counters to add.
+     * @param cp Whether to copy counters or not.
+     */
+    private void addCounters(Iterable<GridHadoopCounter> cntrs, boolean cp) {
+        assert cntrs != null;
+
+        for (GridHadoopCounter cntr : cntrs) {
+            if (cp) {
+                GridHadoopCounter cntrCp = createCounter(cntr.getClass(), cntr.group(), cntr.name());
+
+                cntrCp.merge(cntr);
+
+                cntr = cntrCp;
+            }
+
+            cntrsMap.put(new CounterKey(cntr.getClass(), cntr.group(), cntr.name()), cntr);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends GridHadoopCounter> T counter(String grp, String name, Class<T> cls) {
+        assert cls != null;
+
+        CounterKey mapKey = new CounterKey(cls, grp, name);
+
+        T cntr = (T)cntrsMap.get(mapKey);
+
+        if (cntr == null) {
+            cntr = createCounter(cls, grp, name);
+
+            T old = (T)cntrsMap.putIfAbsent(mapKey, cntr);
+
+            if (old != null)
+                return old;
+        }
+
+        return cntr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridHadoopCounter> all() {
+        return cntrsMap.values();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void merge(GridHadoopCounters other) {
+        for (GridHadoopCounter counter : other.all())
+            counter(counter.group(), counter.name(), counter.getClass()).merge(counter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeCollection(out, cntrsMap.values());
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        addCounters(U.<GridHadoopCounter>readCollection(in), false);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        HadoopCountersImpl counters = (HadoopCountersImpl)o;
+
+        return cntrsMap.equals(counters.cntrsMap);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return cntrsMap.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopCountersImpl.class, this, "counters", cntrsMap.values());
+    }
+
+    /**
+     * The tuple of counter identifier components for more readable code.
+     */
+    private static class CounterKey extends GridTuple3<Class<? extends GridHadoopCounter>, String, String> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Constructor.
+         *
+         * @param cls Class of the counter.
+         * @param grp Group name.
+         * @param name Counter name.
+         */
+        private CounterKey(Class<? extends GridHadoopCounter> cls, String grp, String name) {
+            super(cls, grp, name);
+        }
+
+        /**
+         * Empty constructor required by {@link Externalizable}.
+         */
+        public CounterKey() {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopLongCounter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopLongCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopLongCounter.java
new file mode 100644
index 0000000..1aa1e0e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopLongCounter.java
@@ -0,0 +1,92 @@
+/*
+ * 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.internal.processors.hadoop.*;
+
+import java.io.*;
+
+/**
+ * Standard hadoop counter to use via original Hadoop API in Hadoop jobs.
+ */
+public class HadoopLongCounter extends HadoopCounterAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** The counter value. */
+    private long val;
+
+    /**
+     * Default constructor required by {@link Externalizable}.
+     */
+    public HadoopLongCounter() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param grp Group name.
+     * @param name Counter name.
+     */
+    public HadoopLongCounter(String grp, String name) {
+        super(grp, name);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeValue(ObjectOutput out) throws IOException {
+        out.writeLong(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readValue(ObjectInput in) throws IOException {
+        val = in.readLong();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void merge(GridHadoopCounter cntr) {
+        val += ((HadoopLongCounter)cntr).val;
+    }
+
+    /**
+     * Gets current value of this counter.
+     *
+     * @return Current value.
+     */
+    public long value() {
+        return val;
+    }
+
+    /**
+     * Sets current value by the given value.
+     *
+     * @param val Value to set.
+     */
+    public void value(long val) {
+        this.val = val;
+    }
+
+    /**
+     * Increment this counter by the given value.
+     *
+     * @param i Value to increase this counter by.
+     */
+    public void increment(long i) {
+        val += i;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java
new file mode 100644
index 0000000..f22d0cd
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopPerformanceCounter.java
@@ -0,0 +1,279 @@
+/*
+ * 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.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+
+/**
+ * Counter for the job statistics accumulation.
+ */
+public class HadoopPerformanceCounter extends HadoopCounterAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** The group name for this counter. */
+    private static final String GROUP_NAME = "SYSTEM";
+
+    /** The counter name for this counter. */
+    private static final String COUNTER_NAME = "PERFORMANCE";
+
+    /** Events collections. */
+    private Collection<T2<String,Long>> evts = new ArrayList<>();
+
+    /** Node id to insert into the event info. */
+    private UUID nodeId;
+
+    /** */
+    private int reducerNum;
+
+    /** */
+    private volatile Long firstShuffleMsg;
+
+    /** */
+    private volatile Long lastShuffleMsg;
+
+    /**
+     * Default constructor required by {@link Externalizable}.
+     */
+    public HadoopPerformanceCounter() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param grp Group name.
+     * @param name Counter name.
+     */
+    public HadoopPerformanceCounter(String grp, String name) {
+        super(grp, name);
+    }
+
+    /**
+     * Constructor to create instance to use this as helper.
+     *
+     * @param nodeId Id of the work node.
+     */
+    public HadoopPerformanceCounter(UUID nodeId) {
+        this.nodeId = nodeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeValue(ObjectOutput out) throws IOException {
+        U.writeCollection(out, evts);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void readValue(ObjectInput in) throws IOException {
+        try {
+            evts = U.readCollection(in);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IOException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void merge(GridHadoopCounter cntr) {
+        evts.addAll(((HadoopPerformanceCounter)cntr).evts);
+    }
+
+    /**
+     * Gets the events collection.
+     *
+     * @return Collection of event.
+     */
+    public Collection<T2<String, Long>> evts() {
+        return evts;
+    }
+
+    /**
+     * Generate name that consists of some event information.
+     *
+     * @param info Task info.
+     * @param evtType The type of the event.
+     * @return String contains necessary event information.
+     */
+    private String eventName(GridHadoopTaskInfo info, String evtType) {
+        return eventName(info.type().toString(), info.taskNumber(), evtType);
+    }
+
+    /**
+     * Generate name that consists of some event information.
+     *
+     * @param taskType Task type.
+     * @param taskNum Number of the task.
+     * @param evtType The type of the event.
+     * @return String contains necessary event information.
+     */
+    private String eventName(String taskType, int taskNum, String evtType) {
+        assert nodeId != null;
+
+        return taskType + " " + taskNum + " " + evtType + " " + nodeId;
+    }
+
+    /**
+     * Adds event of the task submission (task instance creation).
+     *
+     * @param info Task info.
+     * @param ts Timestamp of the event.
+     */
+    public void onTaskSubmit(GridHadoopTaskInfo info, long ts) {
+        evts.add(new T2<>(eventName(info, "submit"), ts));
+    }
+
+    /**
+     * Adds event of the task preparation.
+     *
+     * @param info Task info.
+     * @param ts Timestamp of the event.
+     */
+    public void onTaskPrepare(GridHadoopTaskInfo info, long ts) {
+        evts.add(new T2<>(eventName(info, "prepare"), ts));
+    }
+
+    /**
+     * Adds event of the task finish.
+     *
+     * @param info Task info.
+     * @param ts Timestamp of the event.
+     */
+    public void onTaskFinish(GridHadoopTaskInfo info, long ts) {
+        if (info.type() == GridHadoopTaskType.REDUCE && lastShuffleMsg != null) {
+            evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "start"), firstShuffleMsg));
+            evts.add(new T2<>(eventName("SHUFFLE", reducerNum, "finish"), lastShuffleMsg));
+
+            lastShuffleMsg = null;
+        }
+
+        evts.add(new T2<>(eventName(info, "finish"), ts));
+    }
+
+    /**
+     * Adds event of the task run.
+     *
+     * @param info Task info.
+     * @param ts Timestamp of the event.
+     */
+    public void onTaskStart(GridHadoopTaskInfo info, long ts) {
+        evts.add(new T2<>(eventName(info, "start"), ts));
+    }
+
+    /**
+     * Adds event of the job preparation.
+     *
+     * @param ts Timestamp of the event.
+     */
+    public void onJobPrepare(long ts) {
+        assert nodeId != null;
+
+        evts.add(new T2<>("JOB prepare " + nodeId, ts));
+    }
+
+    /**
+     * Adds event of the job start.
+     *
+     * @param ts Timestamp of the event.
+     */
+    public void onJobStart(long ts) {
+        assert nodeId != null;
+
+        evts.add(new T2<>("JOB start " + nodeId, ts));
+    }
+
+    /**
+     * Adds client submission events from job info.
+     *
+     * @param info Job info.
+     */
+    public void clientSubmissionEvents(GridHadoopJobInfo info) {
+        assert nodeId != null;
+
+        addEventFromProperty("JOB requestId", info, REQ_NEW_JOBID_TS_PROPERTY);
+        addEventFromProperty("JOB responseId", info, RESPONSE_NEW_JOBID_TS_PROPERTY);
+        addEventFromProperty("JOB submit", info, JOB_SUBMISSION_START_TS_PROPERTY);
+    }
+
+    /**
+     * Adds event with timestamp from some property in job info.
+     *
+     * @param evt Event type and phase.
+     * @param info Job info.
+     * @param propName Property name to get timestamp.
+     */
+    private void addEventFromProperty(String evt, GridHadoopJobInfo info, String propName) {
+        String val = info.property(propName);
+
+        if (!F.isEmpty(val)) {
+            try {
+                evts.add(new T2<>(evt + " " + nodeId, Long.parseLong(val)));
+            }
+            catch (NumberFormatException e) {
+                throw new IllegalStateException("Invalid value '" + val + "' of property '" + propName + "'", e);
+            }
+        }
+    }
+
+    /**
+     * Registers shuffle message event.
+     *
+     * @param reducerNum Number of reducer that receives the data.
+     * @param ts Timestamp of the event.
+     */
+    public void onShuffleMessage(int reducerNum, long ts) {
+        this.reducerNum = reducerNum;
+
+        if (firstShuffleMsg == null)
+            firstShuffleMsg = ts;
+
+        lastShuffleMsg = ts;
+    }
+
+    /**
+     * Gets system predefined performance counter from the GridHadoopCounters object.
+     *
+     * @param cntrs GridHadoopCounters object.
+     * @param nodeId Node id for methods that adds events. It may be null if you don't use ones.
+     * @return Predefined performance counter.
+     */
+    public static HadoopPerformanceCounter getCounter(GridHadoopCounters cntrs, @Nullable UUID nodeId) {
+        HadoopPerformanceCounter cntr = cntrs.counter(GROUP_NAME, COUNTER_NAME, HadoopPerformanceCounter.class);
+
+        if (nodeId != null)
+            cntr.nodeId(nodeId);
+
+        return cntrs.counter(GROUP_NAME, COUNTER_NAME, HadoopPerformanceCounter.class);
+    }
+
+    /**
+     * Sets the nodeId field.
+     *
+     * @param nodeId Node id.
+     */
+    private void nodeId(UUID nodeId) {
+        this.nodeId = nodeId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopDistributedFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopDistributedFileSystem.java
deleted file mode 100644
index e9461e2..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/GridHadoopDistributedFileSystem.java
+++ /dev/null
@@ -1,91 +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.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
-
-import java.io.*;
-import java.net.*;
-
-import static org.apache.ignite.configuration.IgfsConfiguration.*;
-
-/**
- * Wrapper of HDFS for support of separated working directory.
- */
-public class GridHadoopDistributedFileSystem extends DistributedFileSystem {
-    /** User name for each thread. */
-    private final ThreadLocal<String> userName = new ThreadLocal<String>() {
-        /** {@inheritDoc} */
-        @Override protected String initialValue() {
-            return DFLT_USER_NAME;
-        }
-    };
-
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>() {
-        /** {@inheritDoc} */
-        @Override protected Path initialValue() {
-            return getHomeDirectory();
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public void initialize(URI uri, Configuration conf) throws IOException {
-        super.initialize(uri, conf);
-
-        setUser(conf.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
-    }
-
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(getHomeDirectory());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
-
-        return path.makeQualified(getUri(), null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setWorkingDirectory(Path dir) {
-        Path fixedDir = fixRelativePart(dir);
-
-        String res = fixedDir.toUri().getPath();
-
-        if (!DFSUtil.isValidName(res))
-            throw new IllegalArgumentException("Invalid DFS directory name " + res);
-
-        workingDir.set(fixedDir);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getWorkingDirectory() {
-        return workingDir.get();
-    }
-}


[08/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
index 2be16cb..a2b89a6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopTcpNioCommunicationClient.java
@@ -66,7 +66,7 @@ public class GridHadoopTcpNioCommunicationClient extends GridHadoopAbstractCommu
     }
 
     /** {@inheritDoc} */
-    @Override public void sendMessage(GridHadoopProcessDescriptor desc, GridHadoopMessage msg)
+    @Override public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg)
         throws IgniteCheckedException {
         if (closed())
             throw new IgniteCheckedException("Client was closed: " + this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
index b986d3e..fb10720 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Counter.java
@@ -31,14 +31,14 @@ import static org.apache.hadoop.mapreduce.util.CountersStrings.*;
  */
 public class GridHadoopV1Counter extends Counters.Counter {
     /** Delegate. */
-    private final GridHadoopLongCounter cntr;
+    private final HadoopLongCounter cntr;
 
     /**
      * Creates new instance.
      *
      * @param cntr Delegate counter.
      */
-    public GridHadoopV1Counter(GridHadoopLongCounter cntr) {
+    public GridHadoopV1Counter(HadoopLongCounter cntr) {
         this.cntr = cntr;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
index 1abb2c0..791ccdc 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v1/GridHadoopV1Reporter.java
@@ -49,7 +49,7 @@ public class GridHadoopV1Reporter implements Reporter {
 
     /** {@inheritDoc} */
     @Override public Counters.Counter getCounter(String grp, String name) {
-        return new GridHadoopV1Counter(ctx.counter(grp, name, GridHadoopLongCounter.class));
+        return new GridHadoopV1Counter(ctx.counter(grp, name, HadoopLongCounter.class));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
index 160e34b..9964d91 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Context.java
@@ -171,7 +171,7 @@ public class GridHadoopV2Context extends JobContextImpl implements MapContext, R
 
     /** {@inheritDoc} */
     @Override public Counter getCounter(String grpName, String cntrName) {
-        return new GridHadoopV2Counter(ctx.counter(grpName, cntrName, GridHadoopLongCounter.class));
+        return new GridHadoopV2Counter(ctx.counter(grpName, cntrName, HadoopLongCounter.class));
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
index 6bf8a44..c4fc599 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Counter.java
@@ -27,14 +27,14 @@ import java.io.*;
  */
 public class GridHadoopV2Counter implements Counter {
     /** Delegate. */
-    private final GridHadoopLongCounter cntr;
+    private final HadoopLongCounter cntr;
 
     /**
      * Creates new instance with given delegate.
      *
      * @param cntr Internal counter.
      */
-    public GridHadoopV2Counter(GridHadoopLongCounter cntr) {
+    public GridHadoopV2Counter(HadoopLongCounter cntr) {
         assert cntr != null : "counter must be non-null";
 
         this.cntr = cntr;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
index 5f1af22..5145bdb 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2Job.java
@@ -97,7 +97,7 @@ public class GridHadoopV2Job implements GridHadoopJob {
 
         jobConf = new JobConf();
 
-        GridHadoopFileSystemsUtils.setupFileSystems(jobConf);
+        HadoopFileSystemsUtils.setupFileSystems(jobConf);
 
         Thread.currentThread().setContextClassLoader(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
index be619c7..fdda3f6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2JobResourceManager.java
@@ -82,7 +82,7 @@ public class GridHadoopV2JobResourceManager {
         Thread.currentThread().setContextClassLoader(cfg.getClassLoader());
 
         try {
-            cfg.set(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
+            cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
 
             if(!cfg.getBoolean("fs.file.impl.disable.cache", false))
                 FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
index 3e88362..476d8f0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/GridHadoopV2TaskContext.java
@@ -89,7 +89,7 @@ public class GridHadoopV2TaskContext extends GridHadoopTaskContext {
     private UUID locNodeId;
 
     /** Counters for task. */
-    private final GridHadoopCounters cntrs = new GridHadoopCountersImpl();
+    private final GridHadoopCounters cntrs = new HadoopCountersImpl();
 
     /**
      * @param taskInfo Task info.
@@ -234,7 +234,7 @@ public class GridHadoopV2TaskContext extends GridHadoopTaskContext {
         try {
             FileSystem fs = FileSystem.get(jobConf());
 
-            GridHadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
+            HadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
 
             LocalFileSystem locFs = FileSystem.getLocal(jobConf());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java
index 1db87a0..eeb6509 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopAbstractSelfTest.java
@@ -210,7 +210,7 @@ public abstract class GridHadoopAbstractSelfTest extends GridCommonAbstractTest
         cfg.set("fs.AbstractFileSystem.igfs.impl", IgniteHadoopFileSystem.
             class.getName());
 
-        GridHadoopFileSystemsUtils.setupFileSystems(cfg);
+        HadoopFileSystemsUtils.setupFileSystems(cfg);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
index 7e2b2de..47dc727 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopCommandLineTest.java
@@ -19,9 +19,9 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import com.google.common.base.*;
 import org.apache.ignite.*;
+import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -170,7 +170,7 @@ public class GridHadoopCommandLineTest extends GridCommonAbstractTest {
                     out.println(
                         "    <property>\n" +
                         "        <name>" + HadoopUtils.JOB_COUNTER_WRITER_PROPERTY + "</name>\n" +
-                        "        <value>" + GridHadoopFSCounterWriter.class.getName() + "</value>\n" +
+                        "        <value>" + IgniteHadoopFileSystemCounterWriter.class.getName() + "</value>\n" +
                         "    </property>\n");
 
                 out.println(line);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
deleted file mode 100644
index 947159c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ /dev/null
@@ -1,1005 +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.cache.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.igfs.mapreduce.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.cluster.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.planner.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.testframework.*;
-import org.jetbrains.annotations.*;
-
-import java.net.*;
-import java.util.*;
-
-/**
- *
- */
-public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstractSelfTest {
-    /** */
-    private static final UUID ID_1 = new UUID(0, 1);
-
-    /** */
-    private static final UUID ID_2 = new UUID(0, 2);
-
-    /** */
-    private static final UUID ID_3 = new UUID(0, 3);
-
-    /** */
-    private static final String HOST_1 = "host1";
-
-    /** */
-    private static final String HOST_2 = "host2";
-
-    /** */
-    private static final String HOST_3 = "host3";
-
-    /** */
-    private static final String INVALID_HOST_1 = "invalid_host1";
-
-    /** */
-    private static final String INVALID_HOST_2 = "invalid_host2";
-
-    /** */
-    private static final String INVALID_HOST_3 = "invalid_host3";
-
-    /** Mocked Grid. */
-    private static final MockIgnite GRID = new MockIgnite();
-
-    /** Mocked IGFS. */
-    private static final IgniteFs IGFS = new MockIgfs();
-
-    /** Planner. */
-    private static final GridHadoopMapReducePlanner PLANNER = new GridHadoopDefaultMapReducePlanner();
-
-    /** Block locations. */
-    private static final Map<Block, Collection<IgfsBlockLocation>> BLOCK_MAP = new HashMap<>();
-
-    /** Proxy map. */
-    private static final Map<URI, Boolean> PROXY_MAP = new HashMap<>();
-
-    /** Last created plan. */
-    private static final ThreadLocal<GridHadoopMapReducePlan> PLAN = new ThreadLocal<>();
-
-    /**
-     *
-     */
-    static {
-        GridTestUtils.setFieldValue(PLANNER, "ignite", GRID);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        GridTestUtils.setFieldValue(PLANNER, "log", log());
-
-        BLOCK_MAP.clear();
-        PROXY_MAP.clear();
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsOneBlockPerNode() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1);
-        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_2);
-        GridHadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_2));
-        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 2);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsOneBlockPerNode() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1);
-        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_2);
-        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 2);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2);
-        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2);
-        GridHadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_1, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1, ID_2));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_1, ID_2));
-        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_1, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1, HOST_2);
-        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_1, HOST_2);
-        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_1, HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsSeveralComplexBlocksPerNode() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2, HOST_3);
-        GridHadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_1, ID_3));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_2, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split2);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_1);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) || ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsOrphans() throws IgniteCheckedException {
-        GridHadoopFileBlock split1 = split(false, "/file1", 0, 100, INVALID_HOST_1, INVALID_HOST_2);
-        GridHadoopFileBlock split2 = split(false, "/file2", 0, 100, INVALID_HOST_1, INVALID_HOST_3);
-        GridHadoopFileBlock split3 = split(false, "/file3", 0, 100, INVALID_HOST_2, INVALID_HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 1);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 2);
-
-        plan(1, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 0) ||
-            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 0) ||
-            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 1);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * Create plan.
-     *
-     * @param reducers Reducers count.
-     * @param splits Splits.
-     * @return Plan.
-     * @throws IgniteCheckedException If failed.
-     */
-    private static GridHadoopMapReducePlan plan(int reducers, GridHadoopInputSplit... splits) throws IgniteCheckedException {
-        assert reducers > 0;
-        assert splits != null && splits.length > 0;
-
-        Collection<GridHadoopInputSplit> splitList = new ArrayList<>(splits.length);
-
-        Collections.addAll(splitList, splits);
-
-        Collection<ClusterNode> top = new ArrayList<>();
-
-        GridTestNode node1 = new GridTestNode(ID_1);
-        GridTestNode node2 = new GridTestNode(ID_2);
-        GridTestNode node3 = new GridTestNode(ID_3);
-
-        node1.setHostName(HOST_1);
-        node2.setHostName(HOST_2);
-        node3.setHostName(HOST_3);
-
-        top.add(node1);
-        top.add(node2);
-        top.add(node3);
-
-        GridHadoopMapReducePlan plan = PLANNER.preparePlan(new MockJob(reducers, splitList), top, null);
-
-        PLAN.set(plan);
-
-        return plan;
-    }
-
-    /**
-     * Ensure that node contains the given mappers.
-     *
-     * @param nodeId Node ID.
-     * @param expSplits Expected splits.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureMappers(UUID nodeId, GridHadoopInputSplit... expSplits) {
-        Collection<GridHadoopInputSplit> expSplitsCol = new ArrayList<>();
-
-        Collections.addAll(expSplitsCol, expSplits);
-
-        Collection<GridHadoopInputSplit> splits = PLAN.get().mappers(nodeId);
-
-        return F.eq(expSplitsCol, splits);
-    }
-
-    /**
-     * Ensure that node contains the given amount of reducers.
-     *
-     * @param nodeId Node ID.
-     * @param reducers Reducers.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureReducers(UUID nodeId, int reducers) {
-        int[] reducersArr = PLAN.get().reducers(nodeId);
-
-        return reducers == 0 ? F.isEmpty(reducersArr) : (reducersArr != null && reducersArr.length == reducers);
-    }
-
-    /**
-     * Ensure that no mappers and reducers is located on this node.
-     *
-     * @param nodeId Node ID.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureEmpty(UUID nodeId) {
-        return F.isEmpty(PLAN.get().mappers(nodeId)) && F.isEmpty(PLAN.get().reducers(nodeId));
-    }
-
-    /**
-     * Create split.
-     *
-     * @param igfs IGFS flag.
-     * @param file File.
-     * @param start Start.
-     * @param len Length.
-     * @param hosts Hosts.
-     * @return Split.
-     */
-    private static GridHadoopFileBlock split(boolean igfs, String file, long start, long len, String... hosts) {
-        URI uri = URI.create((igfs ? "igfs://igfs@" : "hdfs://") + file);
-
-        return new GridHadoopFileBlock(hosts, uri, start, len);
-    }
-
-    /**
-     * Create block location.
-     *
-     * @param start Start.
-     * @param len Length.
-     * @param nodeIds Node IDs.
-     * @return Block location.
-     */
-    private static IgfsBlockLocation location(long start, long len, UUID... nodeIds) {
-        assert nodeIds != null && nodeIds.length > 0;
-
-        Collection<ClusterNode> nodes = new ArrayList<>(nodeIds.length);
-
-        for (UUID id : nodeIds)
-            nodes.add(new GridTestNode(id));
-
-        return new IgfsBlockLocationImpl(start, len, nodes);
-    }
-
-    /**
-     * Map IGFS block to nodes.
-     *
-     * @param file File.
-     * @param start Start.
-     * @param len Length.
-     * @param locations Locations.
-     */
-    private static void mapIgfsBlock(URI file, long start, long len, IgfsBlockLocation... locations) {
-        assert locations != null && locations.length > 0;
-
-        IgfsPath path = new IgfsPath(file);
-
-        Block block = new Block(path, start, len);
-
-        Collection<IgfsBlockLocation> locationsList = new ArrayList<>();
-
-        Collections.addAll(locationsList, locations);
-
-        BLOCK_MAP.put(block, locationsList);
-    }
-
-    /**
-     * Block.
-     */
-    private static class Block {
-        /** */
-        private final IgfsPath path;
-
-        /** */
-        private final long start;
-
-        /** */
-        private final long len;
-
-        /**
-         * Constructor.
-         *
-         * @param path Path.
-         * @param start Start.
-         * @param len Length.
-         */
-        private Block(IgfsPath path, long start, long len) {
-            this.path = path;
-            this.start = start;
-            this.len = len;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("RedundantIfStatement")
-        @Override public boolean equals(Object o) {
-            if (this == o) return true;
-            if (!(o instanceof Block)) return false;
-
-            Block block = (Block) o;
-
-            if (len != block.len)
-                return false;
-
-            if (start != block.start)
-                return false;
-
-            if (!path.equals(block.path))
-                return false;
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = path.hashCode();
-
-            res = 31 * res + (int) (start ^ (start >>> 32));
-            res = 31 * res + (int) (len ^ (len >>> 32));
-
-            return res;
-        }
-    }
-
-    /**
-     * Mocked job.
-     */
-    private static class MockJob implements GridHadoopJob {
-        /** Reducers count. */
-        private final int reducers;
-
-        /** */
-        private Collection<GridHadoopInputSplit> splitList;
-
-        /**
-         * Constructor.
-         *
-         * @param reducers Reducers count.
-         * @param splitList Splits.
-         */
-        private MockJob(int reducers, Collection<GridHadoopInputSplit> splitList) {
-            this.reducers = reducers;
-            this.splitList = splitList;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridHadoopJobId id() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridHadoopJobInfo info() {
-            return new HadoopDefaultJobInfo() {
-                @Override public int reducers() {
-                    return reducers;
-                }
-            };
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<GridHadoopInputSplit> input() throws IgniteCheckedException {
-            return splitList;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridHadoopTaskContext getTaskContext(GridHadoopTaskInfo info) throws IgniteCheckedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void initialize(boolean external, UUID nodeId) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void dispose(boolean external) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void prepareTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cleanupTaskEnvironment(GridHadoopTaskInfo info) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cleanupStagingDirectory() {
-            // No-op.
-        }
-    }
-
-    /**
-     * Mocked IGFS.
-     */
-    private static class MockIgfs implements IgfsEx {
-        /** {@inheritDoc} */
-        @Override public boolean isProxy(URI path) {
-            return PROXY_MAP.containsKey(path) && PROXY_MAP.get(path);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) {
-            return BLOCK_MAP.get(new Block(path, start, len));
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len,
-            long maxLen) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void stop() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsContext context() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsPaths proxyPaths() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize, int seqReadsBeforePrefetch) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsInputStreamAdapter open(IgfsPath path) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsInputStreamAdapter open(IgfsPath path, int bufSize) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsStatus globalSpace() throws IgniteCheckedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void globalSampling(@Nullable Boolean val) throws IgniteCheckedException {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Boolean globalSampling() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsLocalMetrics localMetrics() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long groupBlockSize() {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public String clientLogDirectory() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void clientLogDirectory(String logDir) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean evictExclude(IgfsPath path, boolean primary) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public String name() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsConfiguration configuration() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean exists(IgfsPath path) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public IgfsFile info(IgfsPath path) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsPathSummary summary(IgfsPath path) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void rename(IgfsPath src, IgfsPath dest) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean delete(IgfsPath path, boolean recursive) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void mkdirs(IgfsPath path) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public long usedSpaceSize() {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Map<String, String> properties() {
-            return Collections.emptyMap();
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsOutputStream create(IgfsPath path, boolean overwrite) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite, int replication,
-            long blockSize, @Nullable Map<String, String> props) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsOutputStream create(IgfsPath path, int bufSize, boolean overwrite,
-            @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsOutputStream append(IgfsPath path, boolean create) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsOutputStream append(IgfsPath path, int bufSize, boolean create,
-            @Nullable Map<String, String> props) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void setTimes(IgfsPath path, long accessTime, long modificationTime) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgfsMetrics metrics() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void resetMetrics() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public long size(IgfsPath path) {
-            return 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void format() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
-            Collection<IgfsPath> paths, @Nullable T arg) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T, R> R execute(IgfsTask<T, R> task, @Nullable IgfsRecordResolver rslvr,
-            Collection<IgfsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
-            @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, @Nullable T arg) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T, R> R execute(Class<? extends IgfsTask<T, R>> taskCls,
-            @Nullable IgfsRecordResolver rslvr, Collection<IgfsPath> paths, boolean skipNonExistentFiles,
-            long maxRangeLen, @Nullable T arg) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteUuid nextAffinityKey() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteFs withAsync() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isAsync() {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <R> IgniteFuture<R> future() {
-            return null;
-        }
-    }
-
-    /**
-     * Mocked Grid.
-     */
-    @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    private static class MockIgnite extends IgniteSpringBean implements IgniteEx {
-        /** {@inheritDoc} */
-        @Override public IgniteClusterEx cluster() {
-            return (IgniteClusterEx)super.cluster();
-        }
-
-        /** {@inheritDoc} */
-        @Override public IgniteFs igfsx(String name) {
-            assert F.eq("igfs", name);
-
-            return IGFS;
-        }
-
-        /** {@inheritDoc} */
-        @Override public GridHadoop hadoop() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String name() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public <K extends GridCacheUtilityKey, V> GridCacheProjectionEx<K, V> utilityCache(Class<K> keyCls,
-            Class<V> valCls) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public <K, V> GridCache<K, V> cachex(@Nullable String name) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public <K, V> GridCache<K, V> cachex() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public Collection<GridCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super GridCache<?, ?>>... p) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean eventUserRecordable(int type) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean allEventsUserRecordable(int[] types) {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<String> compatibleVersions() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isJmxRemoteEnabled() {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isRestartEnabled() {
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public ClusterNode localNode() {
-            return null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String latestVersion() {
-            return null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java
index 18e5c03..3ebc8ae 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopFileSystemsTest.java
@@ -67,7 +67,7 @@ public class GridHadoopFileSystemsTest extends GridHadoopAbstractSelfTest {
 
         setupFileSystems(cfg);
 
-        cfg.set(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP,
+        cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP,
             new Path(new Path(uri), "user/" + System.getProperty("user.name")).toString());
 
         final CountDownLatch changeUserPhase = new CountDownLatch(THREAD_COUNT);
@@ -89,7 +89,7 @@ public class GridHadoopFileSystemsTest extends GridHadoopAbstractSelfTest {
 
                     FileSystem fs = FileSystem.get(uri, cfg);
 
-                    GridHadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
+                    HadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
 
                     if ("file".equals(uri.getScheme()))
                         FileSystem.get(uri, cfg).setWorkingDirectory(new Path("file:///user/user" + curThreadNum));
@@ -132,7 +132,7 @@ public class GridHadoopFileSystemsTest extends GridHadoopAbstractSelfTest {
 
             Path workDir = new Path(new Path(uri), "user/user" + i);
 
-            cfg.set(GridHadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, workDir.toString());
+            cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, workDir.toString());
 
             assertEquals(workDir, FileSystem.newInstance(uri, cfg).getWorkingDirectory());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
index 7b756da..6bddf9b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.*;
+import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
@@ -68,9 +69,9 @@ public class GridHadoopMapReduceTest extends GridHadoopAbstractWordCountTest {
 
             JobConf jobConf = new JobConf();
 
-            jobConf.set(JOB_COUNTER_WRITER_PROPERTY, GridHadoopFSCounterWriter.class.getName());
+            jobConf.set(JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName());
             jobConf.setUser("yyy");
-            jobConf.set(GridHadoopFSCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
+            jobConf.set(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
 
             //To split into about 40 items for v2
             jobConf.setInt(FileInputFormat.SPLIT_MAXSIZE, 65000);
@@ -123,7 +124,7 @@ public class GridHadoopMapReduceTest extends GridHadoopAbstractWordCountTest {
     private void checkJobStatistics(GridHadoopJobId jobId) throws IgniteCheckedException, IOException {
         GridHadoopCounters cntrs = grid(0).hadoop().counters(jobId);
 
-        GridHadoopPerformanceCounter perfCntr = GridHadoopPerformanceCounter.getCounter(cntrs, null);
+        HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null);
 
         Map<String, SortedMap<Integer,Long>> tasks = new TreeMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestRoundRobinMrPlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestRoundRobinMrPlanner.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestRoundRobinMrPlanner.java
index 5baa8cd..f2b9981 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestRoundRobinMrPlanner.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTestRoundRobinMrPlanner.java
@@ -61,6 +61,6 @@ public class GridHadoopTestRoundRobinMrPlanner implements GridHadoopMapReducePla
         for (int i = 0; i < rdc.length; i++)
             rdc[i] = i;
 
-        return new GridHadoopDefaultMapReducePlan(mappers, Collections.singletonMap(it.next().id(), rdc));
+        return new HadoopDefaultMapReducePlan(mappers, Collections.singletonMap(it.next().id(), rdc));
     }
 }


[11/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java
deleted file mode 100644
index 92854f1..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopHashMultimapBase.java
+++ /dev/null
@@ -1,208 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-
-import java.util.*;
-
-/**
- * Base class for hash multimaps.
- */
-public abstract class GridHadoopHashMultimapBase extends GridHadoopMultimapBase {
-    /**
-     * @param jobInfo Job info.
-     * @param mem Memory.
-     */
-    protected GridHadoopHashMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
-        super(jobInfo, mem);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
-        throw new UnsupportedOperationException("visit");
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        return new Input(taskCtx);
-    }
-
-    /**
-     * @return Hash table capacity.
-     */
-    public abstract int capacity();
-
-    /**
-     * @param idx Index in hash table.
-     * @return Meta page pointer.
-     */
-    protected abstract long meta(int idx);
-
-    /**
-     * @param meta Meta pointer.
-     * @return Key hash.
-     */
-    protected int keyHash(long meta) {
-        return mem.readInt(meta);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Key size.
-     */
-    protected int keySize(long meta) {
-        return mem.readInt(meta + 4);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Key pointer.
-     */
-    protected long key(long meta) {
-        return mem.readLong(meta + 8);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Value pointer.
-     */
-    protected long value(long meta) {
-        return mem.readLong(meta + 16);
-    }
-    /**
-     * @param meta Meta pointer.
-     * @param val Value pointer.
-     */
-    protected void value(long meta, long val) {
-        mem.writeLong(meta + 16, val);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Collision pointer.
-     */
-    protected long collision(long meta) {
-        return mem.readLong(meta + 24);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param collision Collision pointer.
-     */
-    protected void collision(long meta, long collision) {
-        assert meta != collision : meta;
-
-        mem.writeLong(meta + 24, collision);
-    }
-
-    /**
-     * Reader for key and value.
-     */
-    protected class Reader extends ReaderBase {
-        /**
-         * @param ser Serialization.
-         */
-        protected Reader(GridHadoopSerialization ser) {
-            super(ser);
-        }
-
-        /**
-         * @param meta Meta pointer.
-         * @return Key.
-         */
-        public Object readKey(long meta) {
-            assert meta > 0 : meta;
-
-            try {
-                return read(key(meta), keySize(meta));
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-        }
-    }
-
-    /**
-     * Task input.
-     */
-    protected class Input implements GridHadoopTaskInput {
-        /** */
-        private int idx = -1;
-
-        /** */
-        private long metaPtr;
-
-        /** */
-        private final int cap;
-
-        /** */
-        private final Reader keyReader;
-
-        /** */
-        private final Reader valReader;
-
-        /**
-         * @param taskCtx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        public Input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-            cap = capacity();
-
-            keyReader = new Reader(taskCtx.keySerialization());
-            valReader = new Reader(taskCtx.valueSerialization());
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean next() {
-            if (metaPtr != 0) {
-                metaPtr = collision(metaPtr);
-
-                if (metaPtr != 0)
-                    return true;
-            }
-
-            while (++idx < cap) { // Scan table.
-                metaPtr = meta(idx);
-
-                if (metaPtr != 0)
-                    return true;
-            }
-
-            return false;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object key() {
-            return keyReader.readKey(metaPtr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Iterator<?> values() {
-            return new ValueIterator(value(metaPtr), valReader);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            keyReader.close();
-            valReader.close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java
deleted file mode 100644
index b8eb12c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimap.java
+++ /dev/null
@@ -1,112 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Multimap for hadoop intermediate results.
- */
-@SuppressWarnings("PublicInnerClass")
-public interface GridHadoopMultimap extends AutoCloseable {
-    /**
-     * Incrementally visits all the keys and values in the map.
-     *
-     * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning.
-     * @param v Visitor.
-     * @return {@code false} If visiting was impossible.
-     */
-    public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException;
-
-    /**
-     * @param ctx Task context.
-     * @return Adder.
-     * @throws IgniteCheckedException If failed.
-     */
-    public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException;
-
-    /**
-     * @param taskCtx Task context.
-     * @return Task input.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx)
-        throws IgniteCheckedException;
-
-    /** {@inheritDoc} */
-    @Override public void close();
-
-    /**
-     * Adder.
-     */
-    public interface Adder extends GridHadoopTaskOutput {
-        /**
-         * @param in Data input.
-         * @param reuse Reusable key.
-         * @return Key.
-         * @throws IgniteCheckedException If failed.
-         */
-        public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException;
-    }
-
-    /**
-     * Key add values to.
-     */
-    public interface Key {
-        /**
-         * @param val Value.
-         */
-        public void add(Value val);
-    }
-
-    /**
-     * Value.
-     */
-    public interface Value {
-        /**
-         * @return Size in bytes.
-         */
-        public int size();
-
-        /**
-         * @param ptr Pointer.
-         */
-        public void copyTo(long ptr);
-    }
-
-    /**
-     * Key and values visitor.
-     */
-    public interface Visitor {
-        /**
-         * @param keyPtr Key pointer.
-         * @param keySize Key size.
-         */
-        public void onKey(long keyPtr, int keySize) throws IgniteCheckedException;
-
-        /**
-         * @param valPtr Value pointer.
-         * @param valSize Value size.
-         */
-        public void onValue(long valPtr, int valSize) throws IgniteCheckedException;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java
deleted file mode 100644
index 2d8660f..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopMultimapBase.java
+++ /dev/null
@@ -1,368 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.streams.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
-
-/**
- * Base class for all multimaps.
- */
-public abstract class GridHadoopMultimapBase implements GridHadoopMultimap {
-    /** */
-    protected final GridUnsafeMemory mem;
-
-    /** */
-    protected final int pageSize;
-
-    /** */
-    private final Collection<GridLongList> allPages = new ConcurrentLinkedQueue<>();
-
-    /**
-     * @param jobInfo Job info.
-     * @param mem Memory.
-     */
-    protected GridHadoopMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
-        assert jobInfo != null;
-        assert mem != null;
-
-        this.mem = mem;
-
-        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, 32 * 1024);
-    }
-
-    /**
-     * @param ptrs Page pointers.
-     */
-    private void deallocate(GridLongList ptrs) {
-        while (!ptrs.isEmpty())
-            mem.release(ptrs.remove(), ptrs.remove());
-    }
-
-    /**
-     * @param valPtr Value page pointer.
-     * @param nextValPtr Next value page pointer.
-     */
-    protected void nextValue(long valPtr, long nextValPtr) {
-        mem.writeLong(valPtr, nextValPtr);
-    }
-
-    /**
-     * @param valPtr Value page pointer.
-     * @return Next value page pointer.
-     */
-    protected long nextValue(long valPtr) {
-        return mem.readLong(valPtr);
-    }
-
-    /**
-     * @param valPtr Value page pointer.
-     * @param size Size.
-     */
-    protected void valueSize(long valPtr, int size) {
-        mem.writeInt(valPtr + 8, size);
-    }
-
-    /**
-     * @param valPtr Value page pointer.
-     * @return Value size.
-     */
-    protected int valueSize(long valPtr) {
-        return mem.readInt(valPtr + 8);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        for (GridLongList list : allPages)
-            deallocate(list);
-    }
-
-    /**
-     * Reader for key and value.
-     */
-    protected class ReaderBase implements AutoCloseable {
-        /** */
-        private Object tmp;
-
-        /** */
-        private final GridHadoopSerialization ser;
-
-        /** */
-        private final GridHadoopDataInStream in = new GridHadoopDataInStream(mem);
-
-        /**
-         * @param ser Serialization.
-         */
-        protected ReaderBase(GridHadoopSerialization ser) {
-            assert ser != null;
-
-            this.ser = ser;
-        }
-
-        /**
-         * @param valPtr Value page pointer.
-         * @return Value.
-         */
-        public Object readValue(long valPtr) {
-            assert valPtr > 0 : valPtr;
-
-            try {
-                return read(valPtr + 12, valueSize(valPtr));
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-        }
-
-        /**
-         * Resets temporary object to the given one.
-         *
-         * @param tmp Temporary object for reuse.
-         */
-        public void resetReusedObject(Object tmp) {
-            this.tmp = tmp;
-        }
-
-        /**
-         * @param ptr Pointer.
-         * @param size Object size.
-         * @return Object.
-         */
-        protected Object read(long ptr, long size) throws IgniteCheckedException {
-            in.buffer().set(ptr, size);
-
-            tmp = ser.read(in, tmp);
-
-            return tmp;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            ser.close();
-        }
-    }
-
-    /**
-     * Base class for adders.
-     */
-    protected abstract class AdderBase implements Adder {
-        /** */
-        protected final GridHadoopSerialization keySer;
-
-        /** */
-        protected final GridHadoopSerialization valSer;
-
-        /** */
-        private final GridHadoopDataOutStream out;
-
-        /** */
-        private long writeStart;
-
-        /** Size and pointer pairs list. */
-        private final GridLongList pages = new GridLongList(16);
-
-        /**
-         * @param ctx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected AdderBase(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-            valSer = ctx.valueSerialization();
-            keySer = ctx.keySerialization();
-
-            out = new GridHadoopDataOutStream(mem) {
-                @Override public long move(long size) {
-                    long ptr = super.move(size);
-
-                    if (ptr == 0) // Was not able to move - not enough free space.
-                        ptr = allocateNextPage(size);
-
-                    assert ptr != 0;
-
-                    return ptr;
-                }
-            };
-        }
-
-        /**
-         * @param requestedSize Requested size.
-         * @return Next write pointer.
-         */
-        private long allocateNextPage(long requestedSize) {
-            int writtenSize = writtenSize();
-
-            long newPageSize = Math.max(writtenSize + requestedSize, pageSize);
-            long newPagePtr = mem.allocate(newPageSize);
-
-            pages.add(newPageSize);
-            pages.add(newPagePtr);
-
-            GridHadoopOffheapBuffer b = out.buffer();
-
-            b.set(newPagePtr, newPageSize);
-
-            if (writtenSize != 0) {
-                mem.copyMemory(writeStart, newPagePtr, writtenSize);
-
-                b.move(writtenSize);
-            }
-
-            writeStart = newPagePtr;
-
-            return b.move(requestedSize);
-        }
-
-        /**
-         * @return Fixed pointer.
-         */
-        private long fixAlignment() {
-            GridHadoopOffheapBuffer b = out.buffer();
-
-            long ptr = b.pointer();
-
-            if ((ptr & 7L) != 0) { // Address is not aligned by octet.
-                ptr = (ptr + 8L) & ~7L;
-
-                b.pointer(ptr);
-            }
-
-            return ptr;
-        }
-
-        /**
-         * @param off Offset.
-         * @param o Object.
-         * @return Page pointer.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected long write(int off, Object o, GridHadoopSerialization ser) throws IgniteCheckedException {
-            writeStart = fixAlignment();
-
-            if (off != 0)
-                out.move(off);
-
-            ser.write(out, o);
-
-            return writeStart;
-        }
-
-        /**
-         * @param size Size.
-         * @return Pointer.
-         */
-        protected long allocate(int size) {
-            writeStart = fixAlignment();
-
-            out.move(size);
-
-            return writeStart;
-        }
-
-        /**
-         * Rewinds local allocation pointer to the given pointer if possible.
-         *
-         * @param ptr Pointer.
-         */
-        protected void localDeallocate(long ptr) {
-            GridHadoopOffheapBuffer b = out.buffer();
-
-            if (b.isInside(ptr))
-                b.pointer(ptr);
-            else
-                b.reset();
-        }
-
-        /**
-         * @return Written size.
-         */
-        protected int writtenSize() {
-            return (int)(out.buffer().pointer() - writeStart);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            allPages.add(pages);
-
-            keySer.close();
-            valSer.close();
-        }
-    }
-
-    /**
-     * Iterator over values.
-     */
-    protected class ValueIterator implements Iterator<Object> {
-        /** */
-        private long valPtr;
-
-        /** */
-        private final ReaderBase valReader;
-
-        /**
-         * @param valPtr Value page pointer.
-         * @param valReader Value reader.
-         */
-        protected ValueIterator(long valPtr, ReaderBase valReader) {
-            this.valPtr = valPtr;
-            this.valReader = valReader;
-        }
-
-        /**
-         * @param valPtr Head value pointer.
-         */
-        public void head(long valPtr) {
-            this.valPtr = valPtr;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean hasNext() {
-            return valPtr != 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object next() {
-            if (!hasNext())
-                throw new NoSuchElementException();
-
-            Object res = valReader.readValue(valPtr);
-
-            valPtr = nextValue(valPtr);
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void remove() {
-            throw new UnsupportedOperationException();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipList.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipList.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipList.java
deleted file mode 100644
index a2c626c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/GridHadoopSkipList.java
+++ /dev/null
@@ -1,726 +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.shuffle.collections;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Skip list.
- */
-public class GridHadoopSkipList extends GridHadoopMultimapBase {
-    /** */
-    private static final int HEADS_SIZE = 24 + 33 * 8; // Offset + max level is from 0 to 32 inclusive.
-
-    /** Top level. */
-    private final AtomicInteger topLevel = new AtomicInteger(-1);
-
-    /** Heads for all the lists. */
-    private final long heads;
-
-    /** */
-    private final AtomicBoolean visitGuard = new AtomicBoolean();
-
-    /**
-     * @param jobInfo Job info.
-     * @param mem Memory.
-     */
-    public GridHadoopSkipList(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
-        super(jobInfo, mem);
-
-        heads = mem.allocate(HEADS_SIZE, true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        super.close();
-
-        mem.release(heads, HEADS_SIZE);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
-        if (!visitGuard.compareAndSet(false, true))
-            return false;
-
-        for (long meta = nextMeta(heads, 0); meta != 0L; meta = nextMeta(meta, 0)) {
-            long valPtr = value(meta);
-
-            long lastVisited = ignoreLastVisited ? 0 : lastVisitedValue(meta);
-
-            if (valPtr != lastVisited) {
-                long k = key(meta);
-
-                v.onKey(k + 4, keySize(k));
-
-                lastVisitedValue(meta, valPtr); // Set it to the first value in chain.
-
-                do {
-                    v.onValue(valPtr + 12, valueSize(valPtr));
-
-                    valPtr = nextValue(valPtr);
-                }
-                while (valPtr != lastVisited);
-            }
-        }
-
-        visitGuard.lazySet(false);
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-        return new AdderImpl(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-        Input in = new Input(taskCtx);
-
-        Comparator<Object> grpCmp = taskCtx.groupComparator();
-
-        if (grpCmp != null)
-            return new GroupedInput(grpCmp, in);
-
-        return in;
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Key pointer.
-     */
-    private long key(long meta) {
-        return mem.readLong(meta);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param key Key pointer.
-     */
-    private void key(long meta, long key) {
-        mem.writeLong(meta, key);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Value pointer.
-     */
-    private long value(long meta) {
-        return mem.readLongVolatile(meta + 8);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param valPtr Value pointer.
-     */
-    private void value(long meta, long valPtr) {
-        mem.writeLongVolatile(meta + 8, valPtr);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param oldValPtr Old first value pointer.
-     * @param newValPtr New first value pointer.
-     * @return {@code true} If operation succeeded.
-     */
-    private boolean casValue(long meta, long oldValPtr, long newValPtr) {
-        return mem.casLong(meta + 8, oldValPtr, newValPtr);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @return Last visited value pointer.
-     */
-    private long lastVisitedValue(long meta) {
-        return mem.readLong(meta + 16);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param valPtr Last visited value pointer.
-     */
-    private void lastVisitedValue(long meta, long valPtr) {
-        mem.writeLong(meta + 16, valPtr);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param level Level.
-     * @return Next meta pointer.
-     */
-    private long nextMeta(long meta, int level) {
-        assert meta > 0 : meta;
-
-        return mem.readLongVolatile(meta + 24 + 8 * level);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param level Level.
-     * @param oldNext Old next meta pointer.
-     * @param newNext New next meta pointer.
-     * @return {@code true} If operation succeeded.
-     */
-    private boolean casNextMeta(long meta, int level, long oldNext, long newNext) {
-        assert meta > 0 : meta;
-
-        return mem.casLong(meta + 24 + 8 * level, oldNext, newNext);
-    }
-
-    /**
-     * @param meta Meta pointer.
-     * @param level Level.
-     * @param nextMeta Next meta.
-     */
-    private void nextMeta(long meta, int level, long nextMeta) {
-        assert meta != 0;
-
-        mem.writeLong(meta + 24 + 8 * level, nextMeta);
-    }
-
-    /**
-     * @param keyPtr Key pointer.
-     * @return Key size.
-     */
-    private int keySize(long keyPtr) {
-        return mem.readInt(keyPtr);
-    }
-
-    /**
-     * @param keyPtr Key pointer.
-     * @param keySize Key size.
-     */
-    private void keySize(long keyPtr, int keySize) {
-        mem.writeInt(keyPtr, keySize);
-    }
-
-    /**
-     * @param rnd Random.
-     * @return Next level.
-     */
-    public static int randomLevel(Random rnd) {
-        int x = rnd.nextInt();
-
-        int level = 0;
-
-        while ((x & 1) != 0) { // Count sequential 1 bits.
-            level++;
-
-            x >>>= 1;
-        }
-
-        return level;
-    }
-
-    /**
-     * Reader.
-     */
-    private class Reader extends ReaderBase {
-        /**
-         * @param ser Serialization.
-         */
-        protected Reader(GridHadoopSerialization ser) {
-            super(ser);
-        }
-
-        /**
-         * @param meta Meta pointer.
-         * @return Key.
-         */
-        public Object readKey(long meta) {
-            assert meta > 0 : meta;
-
-            long k = key(meta);
-
-            try {
-                return read(k + 4, keySize(k));
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
-        }
-    }
-
-    /**
-     * Adder.
-     */
-    private class AdderImpl extends AdderBase {
-        /** */
-        private final Comparator<Object> cmp;
-
-        /** */
-        private final Random rnd = new GridRandom();
-
-        /** */
-        private final GridLongList stack = new GridLongList(16);
-
-        /** */
-        private final Reader keyReader;
-
-        /**
-         * @param ctx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        protected AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-            super(ctx);
-
-            keyReader = new Reader(keySer);
-
-            cmp = ctx.sortComparator();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(Object key, Object val) throws IgniteCheckedException {
-            A.notNull(val, "val");
-
-            add(key, val);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
-            KeyImpl k = reuse == null ? new KeyImpl() : (KeyImpl)reuse;
-
-            k.tmpKey = keySer.read(in, k.tmpKey);
-
-            k.meta = add(k.tmpKey, null);
-
-            return k;
-        }
-
-        /**
-         * @param key Key.
-         * @param val Value.
-         * @param level Level.
-         * @return Meta pointer.
-         */
-        private long createMeta(long key, long val, int level) {
-            int size = 32 + 8 * level;
-
-            long meta = allocate(size);
-
-            key(meta, key);
-            value(meta, val);
-            lastVisitedValue(meta, 0L);
-
-            for (int i = 32; i < size; i += 8) // Fill with 0.
-                mem.writeLong(meta + i, 0L);
-
-            return meta;
-        }
-
-        /**
-         * @param key Key.
-         * @return Pointer.
-         * @throws IgniteCheckedException If failed.
-         */
-        private long writeKey(Object key) throws IgniteCheckedException {
-            long keyPtr = write(4, key, keySer);
-            int keySize = writtenSize() - 4;
-
-            keySize(keyPtr, keySize);
-
-            return keyPtr;
-        }
-
-        /**
-         * @param prevMeta Previous meta.
-         * @param meta Next meta.
-         */
-        private void stackPush(long prevMeta, long meta) {
-            stack.add(prevMeta);
-            stack.add(meta);
-        }
-
-        /**
-         * Drops last remembered frame from the stack.
-         */
-        private void stackPop() {
-            stack.pop(2);
-        }
-
-        /**
-         * @param key Key.
-         * @param val Value.
-         * @return Meta pointer.
-         * @throws IgniteCheckedException If failed.
-         */
-        private long add(Object key, @Nullable Object val) throws IgniteCheckedException {
-            assert key != null;
-
-            stack.clear();
-
-            long valPtr = 0;
-
-            if (val != null) { // Write value.
-                valPtr = write(12, val, valSer);
-                int valSize = writtenSize() - 12;
-
-                nextValue(valPtr, 0);
-                valueSize(valPtr, valSize);
-            }
-
-            long keyPtr = 0;
-            long newMeta = 0;
-            int newMetaLevel = -1;
-
-            long prevMeta = heads;
-            int level = topLevel.get();
-            long meta = level < 0 ? 0 : nextMeta(heads, level);
-
-            for (;;) {
-                if (level < 0) { // We did not find our key, trying to add new meta.
-                    if (keyPtr == 0) { // Write key and create meta only once.
-                        keyPtr = writeKey(key);
-
-                        newMetaLevel = randomLevel(rnd);
-                        newMeta = createMeta(keyPtr, valPtr, newMetaLevel);
-                    }
-
-                    nextMeta(newMeta, 0, meta); // Set next to new meta before publishing.
-
-                    if (casNextMeta(prevMeta, 0, meta, newMeta)) { // New key was added successfully.
-                        laceUp(key, newMeta, newMetaLevel);
-
-                        return newMeta;
-                    }
-                    else { // Add failed, need to check out what was added by another thread.
-                        meta = nextMeta(prevMeta, level = 0);
-
-                        stackPop();
-                    }
-                }
-
-                int cmpRes = cmp(key, meta);
-
-                if (cmpRes == 0) { // Key found.
-                    if (newMeta != 0)  // Deallocate if we've allocated something.
-                        localDeallocate(keyPtr);
-
-                    if (valPtr == 0) // Only key needs to be added.
-                        return meta;
-
-                    for (;;) { // Add value for the key found.
-                        long nextVal = value(meta);
-
-                        nextValue(valPtr, nextVal);
-
-                        if (casValue(meta, nextVal, valPtr))
-                            return meta;
-                    }
-                }
-
-                assert cmpRes != 0;
-
-                if (cmpRes > 0) { // Go right.
-                    prevMeta = meta;
-                    meta = nextMeta(meta, level);
-
-                    if (meta != 0) // If nothing to the right then go down.
-                        continue;
-                }
-
-                while (--level >= 0) { // Go down.
-                    stackPush(prevMeta, meta); // Remember the path.
-
-                    long nextMeta = nextMeta(prevMeta, level);
-
-                    if (nextMeta != meta) { // If the meta is the same as on upper level go deeper.
-                        meta = nextMeta;
-
-                        assert meta != 0;
-
-                        break;
-                    }
-                }
-            }
-        }
-
-        /**
-         * @param key Key.
-         * @param meta Meta pointer.
-         * @return Comparison result.
-         */
-        @SuppressWarnings("unchecked")
-        private int cmp(Object key, long meta) {
-            assert meta != 0;
-
-            return cmp.compare(key, keyReader.readKey(meta));
-        }
-
-        /**
-         * Adds appropriate index links between metas.
-         *
-         * @param newMeta Just added meta.
-         * @param newMetaLevel New level.
-         */
-        private void laceUp(Object key, long newMeta, int newMetaLevel) {
-            for (int level = 1; level <= newMetaLevel; level++) { // Go from the bottom up.
-                long prevMeta = heads;
-                long meta = 0;
-
-                if (!stack.isEmpty()) { // Get the path back.
-                    meta = stack.remove();
-                    prevMeta = stack.remove();
-                }
-
-                for (;;) {
-                    nextMeta(newMeta, level, meta);
-
-                    if (casNextMeta(prevMeta, level, meta, newMeta))
-                        break;
-
-                    long oldMeta = meta;
-
-                    meta = nextMeta(prevMeta, level); // Reread meta.
-
-                    for (;;) {
-                        int cmpRes = cmp(key, meta);
-
-                        if (cmpRes > 0) { // Go right.
-                            prevMeta = meta;
-                            meta = nextMeta(prevMeta, level);
-
-                            if (meta != oldMeta) // Old meta already known to be greater than ours or is 0.
-                                continue;
-                        }
-
-                        assert cmpRes != 0; // Two different metas with equal keys must be impossible.
-
-                        break; // Retry cas.
-                    }
-                }
-            }
-
-            if (!stack.isEmpty())
-                return; // Our level already lower than top.
-
-            for (;;) { // Raise top level.
-                int top = topLevel.get();
-
-                if (newMetaLevel <= top || topLevel.compareAndSet(top, newMetaLevel))
-                    break;
-            }
-        }
-
-        /**
-         * Key.
-         */
-        private class KeyImpl implements Key {
-            /** */
-            private long meta;
-
-            /** */
-            private Object tmpKey;
-
-            /**
-             * @return Meta pointer for the key.
-             */
-            public long address() {
-                return meta;
-            }
-
-            /**
-             * @param val Value.
-             */
-            @Override public void add(Value val) {
-                int size = val.size();
-
-                long valPtr = allocate(size + 12);
-
-                val.copyTo(valPtr + 12);
-
-                valueSize(valPtr, size);
-
-                long nextVal;
-
-                do {
-                    nextVal = value(meta);
-
-                    nextValue(valPtr, nextVal);
-                }
-                while(!casValue(meta, nextVal, valPtr));
-            }
-        }
-    }
-
-    /**
-     * Task input.
-     */
-    private class Input implements GridHadoopTaskInput {
-        /** */
-        private long metaPtr = heads;
-
-        /** */
-        private final Reader keyReader;
-
-        /** */
-        private final Reader valReader;
-
-        /**
-         * @param taskCtx Task context.
-         * @throws IgniteCheckedException If failed.
-         */
-        private Input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
-            keyReader = new Reader(taskCtx.keySerialization());
-            valReader = new Reader(taskCtx.valueSerialization());
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean next() {
-            metaPtr = nextMeta(metaPtr, 0);
-
-            return metaPtr != 0;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object key() {
-            return keyReader.readKey(metaPtr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Iterator<?> values() {
-            return new ValueIterator(value(metaPtr), valReader);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            keyReader.close();
-            valReader.close();
-        }
-    }
-
-    /**
-     * Grouped input using grouping comparator.
-     */
-    private class GroupedInput implements GridHadoopTaskInput {
-        /** */
-        private final Comparator<Object> grpCmp;
-
-        /** */
-        private final Input in;
-
-        /** */
-        private Object prevKey;
-
-        /** */
-        private Object nextKey;
-
-        /** */
-        private final GridLongList vals = new GridLongList();
-
-        /**
-         * @param grpCmp Grouping comparator.
-         * @param in Input.
-         */
-        private GroupedInput(Comparator<Object> grpCmp, Input in) {
-            this.grpCmp = grpCmp;
-            this.in = in;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean next() {
-            if (prevKey == null) { // First call.
-                if (!in.next())
-                    return false;
-
-                prevKey = in.key();
-
-                assert prevKey != null;
-
-                in.keyReader.resetReusedObject(null); // We need 2 instances of key object for comparison.
-
-                vals.add(value(in.metaPtr));
-            }
-            else {
-                if (in.metaPtr == 0) // We reached the end of the input.
-                    return false;
-
-                vals.clear();
-
-                vals.add(value(in.metaPtr));
-
-                in.keyReader.resetReusedObject(prevKey); // Switch key instances.
-
-                prevKey = nextKey;
-            }
-
-            while (in.next()) { // Fill with head value pointers with equal keys.
-                if (grpCmp.compare(prevKey, nextKey = in.key()) == 0)
-                    vals.add(value(in.metaPtr));
-                else
-                    break;
-            }
-
-            assert !vals.isEmpty();
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Object key() {
-            return prevKey;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Iterator<?> values() {
-            assert !vals.isEmpty();
-
-            final ValueIterator valIter = new ValueIterator(vals.get(0), in.valReader);
-
-            return new Iterator<Object>() {
-                /** */
-                private int idx;
-
-                @Override public boolean hasNext() {
-                    if (!valIter.hasNext()) {
-                        if (++idx == vals.size())
-                            return false;
-
-                        valIter.head(vals.get(idx));
-
-                        assert valIter.hasNext();
-                    }
-
-                    return true;
-                }
-
-                @Override public Object next() {
-                    return valIter.next();
-                }
-
-                @Override public void remove() {
-                    valIter.remove();
-                }
-            };
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IgniteCheckedException {
-            in.close();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimap.java
new file mode 100644
index 0000000..46d8bc9
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopConcurrentHashMultimap.java
@@ -0,0 +1,611 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Multimap for map reduce intermediate results.
+ */
+public class HadoopConcurrentHashMultimap extends HadoopHashMultimapBase {
+    /** */
+    private final AtomicReference<State> state = new AtomicReference<>(State.READING_WRITING);
+
+    /** */
+    private volatile AtomicLongArray oldTbl;
+
+    /** */
+    private volatile AtomicLongArray newTbl;
+
+    /** */
+    private final AtomicInteger keys = new AtomicInteger();
+
+    /** */
+    private final CopyOnWriteArrayList<AdderImpl> adders = new CopyOnWriteArrayList<>();
+
+    /** */
+    private final AtomicInteger inputs = new AtomicInteger();
+
+    /**
+     * @param jobInfo Job info.
+     * @param mem Memory.
+     * @param cap Initial capacity.
+     */
+    public HadoopConcurrentHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) {
+        super(jobInfo, mem);
+
+        assert U.isPow2(cap);
+
+        newTbl = oldTbl = new AtomicLongArray(cap);
+    }
+
+    /**
+     * @return Number of keys.
+     */
+    public long keys() {
+        int res = keys.get();
+
+        for (AdderImpl adder : adders)
+            res += adder.locKeys.get();
+
+        return res;
+    }
+
+    /**
+     * @return Current table capacity.
+     */
+    @Override public int capacity() {
+        return oldTbl.length();
+    }
+
+    /**
+     * @return Adder object.
+     * @param ctx Task context.
+     */
+    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+        if (inputs.get() != 0)
+            throw new IllegalStateException("Active inputs.");
+
+        if (state.get() == State.CLOSING)
+            throw new IllegalStateException("Closed.");
+
+        return new AdderImpl(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        assert inputs.get() == 0 : inputs.get();
+        assert adders.isEmpty() : adders.size();
+
+        state(State.READING_WRITING, State.CLOSING);
+
+        if (keys() == 0)
+            return;
+
+        super.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long meta(int idx) {
+        return oldTbl.get(idx);
+    }
+
+    /**
+     * Incrementally visits all the keys and values in the map.
+     *
+     * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning.
+     * @param v Visitor.
+     * @return {@code false} If visiting was impossible due to rehashing.
+     */
+    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
+        if (!state.compareAndSet(State.READING_WRITING, State.VISITING)) {
+            assert state.get() != State.CLOSING;
+
+            return false; // Can not visit while rehashing happens.
+        }
+
+        AtomicLongArray tbl0 = oldTbl;
+
+        for (int i = 0; i < tbl0.length(); i++) {
+            long meta = tbl0.get(i);
+
+            while (meta != 0) {
+                long valPtr = value(meta);
+
+                long lastVisited = ignoreLastVisited ? 0 : lastVisitedValue(meta);
+
+                if (valPtr != lastVisited) {
+                    v.onKey(key(meta), keySize(meta));
+
+                    lastVisitedValue(meta, valPtr); // Set it to the first value in chain.
+
+                    do {
+                        v.onValue(valPtr + 12, valueSize(valPtr));
+
+                        valPtr = nextValue(valPtr);
+                    }
+                    while (valPtr != lastVisited);
+                }
+
+                meta = collision(meta);
+            }
+        }
+
+        state(State.VISITING, State.READING_WRITING);
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        inputs.incrementAndGet();
+
+        if (!adders.isEmpty())
+            throw new IllegalStateException("Active adders.");
+
+        State s = state.get();
+
+        if (s == State.CLOSING)
+            throw new IllegalStateException("Closed.");
+
+        assert s != State.REHASHING;
+
+        return new Input(taskCtx) {
+            @Override public void close() throws IgniteCheckedException {
+                if (inputs.decrementAndGet() < 0)
+                    throw new IllegalStateException();
+
+                super.close();
+            }
+        };
+    }
+
+    /**
+     * @param fromTbl Table.
+     */
+    private void rehashIfNeeded(AtomicLongArray fromTbl) {
+        if (fromTbl.length() == Integer.MAX_VALUE)
+            return;
+
+        long keys0 = keys();
+
+        if (keys0 < 3 * (fromTbl.length() >>> 2)) // New size has to be >= than 3/4 of capacity to rehash.
+            return;
+
+        if (fromTbl != newTbl) // Check if someone else have done the job.
+            return;
+
+        if (!state.compareAndSet(State.READING_WRITING, State.REHASHING)) {
+            assert state.get() != State.CLOSING; // Visiting is allowed, but we will not rehash.
+
+            return;
+        }
+
+        if (fromTbl != newTbl) { // Double check.
+            state(State.REHASHING, State.READING_WRITING); // Switch back.
+
+            return;
+        }
+
+        // Calculate new table capacity.
+        int newLen = fromTbl.length();
+
+        do {
+            newLen <<= 1;
+        }
+        while (newLen < keys0);
+
+        if (keys0 >= 3 * (newLen >>> 2)) // Still more than 3/4.
+            newLen <<= 1;
+
+        // This is our target table for rehashing.
+        AtomicLongArray toTbl = new AtomicLongArray(newLen);
+
+        // Make the new table visible before rehashing.
+        newTbl = toTbl;
+
+        // Rehash.
+        int newMask = newLen - 1;
+
+        long failedMeta = 0;
+
+        GridLongList collisions = new GridLongList(16);
+
+        for (int i = 0; i < fromTbl.length(); i++) { // Scan source table.
+            long meta = fromTbl.get(i);
+
+            assert meta != -1;
+
+            if (meta == 0) { // No entry.
+                failedMeta = 0;
+
+                if (!fromTbl.compareAndSet(i, 0, -1)) // Mark as moved.
+                    i--; // Retry.
+
+                continue;
+            }
+
+            do { // Collect all the collisions before the last one failed to nullify or 0.
+                collisions.add(meta);
+
+                meta = collision(meta);
+            }
+            while (meta != failedMeta);
+
+            do { // Go from the last to the first to avoid 'in-flight' state for meta entries.
+                meta = collisions.remove();
+
+                int addr = keyHash(meta) & newMask;
+
+                for (;;) { // Move meta entry to the new table.
+                    long toCollision = toTbl.get(addr);
+
+                    collision(meta, toCollision);
+
+                    if (toTbl.compareAndSet(addr, toCollision, meta))
+                        break;
+                }
+            }
+            while (!collisions.isEmpty());
+
+            // Here 'meta' will be a root pointer in old table.
+            if (!fromTbl.compareAndSet(i, meta, -1)) { // Try to mark as moved.
+                failedMeta = meta;
+
+                i--; // Retry the same address in table because new keys were added.
+            }
+            else
+                failedMeta = 0;
+        }
+
+        // Now old and new tables will be the same again.
+        oldTbl = toTbl;
+
+        state(State.REHASHING, State.READING_WRITING);
+    }
+
+    /**
+     * Switch state.
+     *
+     * @param oldState Expected state.
+     * @param newState New state.
+     */
+    private void state(State oldState, State newState) {
+        if (!state.compareAndSet(oldState, newState))
+            throw new IllegalStateException();
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Value pointer.
+     */
+    @Override protected long value(long meta) {
+        return mem.readLongVolatile(meta + 16);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param oldValPtr Old value.
+     * @param newValPtr New value.
+     * @return {@code true} If succeeded.
+     */
+    private boolean casValue(long meta, long oldValPtr, long newValPtr) {
+        return mem.casLong(meta + 16, oldValPtr, newValPtr);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Collision pointer.
+     */
+    @Override protected long collision(long meta) {
+        return mem.readLongVolatile(meta + 24);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param collision Collision pointer.
+     */
+    @Override protected void collision(long meta, long collision) {
+        assert meta != collision : meta;
+
+        mem.writeLongVolatile(meta + 24, collision);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Last visited value pointer.
+     */
+    private long lastVisitedValue(long meta) {
+        return mem.readLong(meta + 32);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param valPtr Last visited value pointer.
+     */
+    private void lastVisitedValue(long meta, long valPtr) {
+        mem.writeLong(meta + 32, valPtr);
+    }
+
+    /**
+     * Adder. Must not be shared between threads.
+     */
+    private class AdderImpl extends AdderBase {
+        /** */
+        private final Reader keyReader;
+
+        /** */
+        private final AtomicInteger locKeys = new AtomicInteger();
+
+        /** */
+        private final Random rnd = new GridRandom();
+
+        /**
+         * @param ctx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        private AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+            super(ctx);
+
+            keyReader = new Reader(keySer);
+
+            rehashIfNeeded(oldTbl);
+
+            adders.add(this);
+        }
+
+        /**
+         * @param in Data input.
+         * @param reuse Reusable key.
+         * @return Key.
+         * @throws IgniteCheckedException If failed.
+         */
+        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
+            KeyImpl k = reuse == null ? new KeyImpl() : (KeyImpl)reuse;
+
+            k.tmpKey = keySer.read(in, k.tmpKey);
+
+            k.meta = add(k.tmpKey, null);
+
+            return k;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object key, Object val) throws IgniteCheckedException {
+            A.notNull(val, "val");
+
+            add(key, val);
+        }
+
+        /**
+         * @param tbl Table.
+         */
+        private void incrementKeys(AtomicLongArray tbl) {
+            locKeys.lazySet(locKeys.get() + 1);
+
+            if (rnd.nextInt(tbl.length()) < 512)
+                rehashIfNeeded(tbl);
+        }
+
+        /**
+         * @param keyHash Key hash.
+         * @param keySize Key size.
+         * @param keyPtr Key pointer.
+         * @param valPtr Value page pointer.
+         * @param collisionPtr Pointer to meta with hash collision.
+         * @param lastVisitedVal Last visited value pointer.
+         * @return Created meta page pointer.
+         */
+        private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr, long lastVisitedVal) {
+            long meta = allocate(40);
+
+            mem.writeInt(meta, keyHash);
+            mem.writeInt(meta + 4, keySize);
+            mem.writeLong(meta + 8, keyPtr);
+            mem.writeLong(meta + 16, valPtr);
+            mem.writeLong(meta + 24, collisionPtr);
+            mem.writeLong(meta + 32, lastVisitedVal);
+
+            return meta;
+        }
+
+        /**
+         * @param key Key.
+         * @param val Value.
+         * @return Updated or created meta page pointer.
+         * @throws IgniteCheckedException If failed.
+         */
+        private long add(Object key, @Nullable Object val) throws IgniteCheckedException {
+            AtomicLongArray tbl = oldTbl;
+
+            int keyHash = U.hash(key.hashCode());
+
+            long newMetaPtr = 0;
+
+            long valPtr = 0;
+
+            if (val != null) {
+                valPtr = write(12, val, valSer);
+                int valSize = writtenSize() - 12;
+
+                valueSize(valPtr, valSize);
+            }
+
+            for (AtomicLongArray old = null;;) {
+                int addr = keyHash & (tbl.length() - 1);
+
+                long metaPtrRoot = tbl.get(addr); // Read root meta pointer at this address.
+
+                if (metaPtrRoot == -1) { // The cell was already moved by rehashing.
+                    AtomicLongArray n = newTbl; // Need to read newTbl first here.
+                    AtomicLongArray o = oldTbl;
+
+                    tbl = tbl == o ? n : o; // Trying to get the oldest table but newer than ours.
+
+                    old = null;
+
+                    continue;
+                }
+
+                if (metaPtrRoot != 0) { // Not empty slot.
+                    long metaPtr = metaPtrRoot;
+
+                    do { // Scan all the collisions.
+                        if (keyHash(metaPtr) == keyHash && key.equals(keyReader.readKey(metaPtr))) { // Found key.
+                            if (newMetaPtr != 0)  // Deallocate new meta if one was allocated.
+                                localDeallocate(key(newMetaPtr)); // Key was allocated first, so rewind to it's pointer.
+
+                            if (valPtr != 0) { // Add value if it exists.
+                                long nextValPtr;
+
+                                // Values are linked to each other to a stack like structure.
+                                // Replace the last value in meta with ours and link it as next.
+                                do {
+                                    nextValPtr = value(metaPtr);
+
+                                    nextValue(valPtr, nextValPtr);
+                                }
+                                while (!casValue(metaPtr, nextValPtr, valPtr));
+                            }
+
+                            return metaPtr;
+                        }
+
+                        metaPtr = collision(metaPtr);
+                    }
+                    while (metaPtr != 0);
+
+                    // Here we did not find our key, need to check if it was moved by rehashing to the new table.
+                    if (old == null) { // If the old table already set, then we will just try to update it.
+                        AtomicLongArray n = newTbl;
+
+                        if (n != tbl) { // Rehashing happens, try to find the key in new table but preserve the old one.
+                            old = tbl;
+                            tbl = n;
+
+                            continue;
+                        }
+                    }
+                }
+
+                if (old != null) { // We just checked new table but did not find our key as well as in the old one.
+                    tbl = old; // Try to add new key to the old table.
+
+                    addr = keyHash & (tbl.length() - 1);
+
+                    old = null;
+                }
+
+                if (newMetaPtr == 0) { // Allocate new meta page.
+                    long keyPtr = write(0, key, keySer);
+                    int keySize = writtenSize();
+
+                    if (valPtr != 0)
+                        nextValue(valPtr, 0);
+
+                    newMetaPtr = createMeta(keyHash, keySize, keyPtr, valPtr, metaPtrRoot, 0);
+                }
+                else // Update new meta with root pointer collision.
+                    collision(newMetaPtr, metaPtrRoot);
+
+                if (tbl.compareAndSet(addr, metaPtrRoot, newMetaPtr)) { // Try to replace root pointer with new one.
+                    incrementKeys(tbl);
+
+                    return newMetaPtr;
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            if (!adders.remove(this))
+                throw new IllegalStateException();
+
+            keys.addAndGet(locKeys.get()); // Here we have race and #keys() method can return wrong result but it is ok.
+
+            super.close();
+        }
+
+        /**
+         * Key.
+         */
+        private class KeyImpl implements Key {
+            /** */
+            private long meta;
+
+            /** */
+            private Object tmpKey;
+
+            /**
+             * @return Meta pointer for the key.
+             */
+            public long address() {
+                return meta;
+            }
+
+            /**
+             * @param val Value.
+             */
+            @Override public void add(Value val) {
+                int size = val.size();
+
+                long valPtr = allocate(size + 12);
+
+                val.copyTo(valPtr + 12);
+
+                valueSize(valPtr, size);
+
+                long nextVal;
+
+                do {
+                    nextVal = value(meta);
+
+                    nextValue(valPtr, nextVal);
+                }
+                while(!casValue(meta, nextVal, valPtr));
+            }
+        }
+    }
+
+    /**
+     * Current map state.
+     */
+    private enum State {
+        /** */
+        REHASHING,
+
+        /** */
+        VISITING,
+
+        /** */
+        READING_WRITING,
+
+        /** */
+        CLOSING
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimap.java
new file mode 100644
index 0000000..15b93c6
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimap.java
@@ -0,0 +1,174 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+/**
+ * Hash multimap.
+ */
+public class HadoopHashMultimap extends HadoopHashMultimapBase {
+    /** */
+    private long[] tbl;
+
+    /** */
+    private int keys;
+
+    /**
+     * @param jobInfo Job info.
+     * @param mem Memory.
+     * @param cap Initial capacity.
+     */
+    public HadoopHashMultimap(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem, int cap) {
+        super(jobInfo, mem);
+
+        assert U.isPow2(cap) : cap;
+
+        tbl = new long[cap];
+    }
+
+    /** {@inheritDoc} */
+    @Override public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+        return new AdderImpl(ctx);
+    }
+
+    /**
+     * Rehash.
+     */
+    private void rehash() {
+        long[] newTbl = new long[tbl.length << 1];
+
+        int newMask = newTbl.length - 1;
+
+        for (long meta : tbl) {
+            while (meta != 0) {
+                long collision = collision(meta);
+
+                int idx = keyHash(meta) & newMask;
+
+                collision(meta, newTbl[idx]);
+
+                newTbl[idx] = meta;
+
+                meta = collision;
+            }
+        }
+
+        tbl = newTbl;
+    }
+
+    /**
+     * @return Keys count.
+     */
+    public int keys() {
+        return keys;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int capacity() {
+        return tbl.length;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long meta(int idx) {
+        return tbl[idx];
+    }
+
+    /**
+     * Adder.
+     */
+    private class AdderImpl extends AdderBase {
+        /** */
+        private final Reader keyReader;
+
+        /**
+         * @param ctx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected AdderImpl(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+            super(ctx);
+
+            keyReader = new Reader(keySer);
+        }
+
+        /**
+         * @param keyHash Key hash.
+         * @param keySize Key size.
+         * @param keyPtr Key pointer.
+         * @param valPtr Value page pointer.
+         * @param collisionPtr Pointer to meta with hash collision.
+         * @return Created meta page pointer.
+         */
+        private long createMeta(int keyHash, int keySize, long keyPtr, long valPtr, long collisionPtr) {
+            long meta = allocate(32);
+
+            mem.writeInt(meta, keyHash);
+            mem.writeInt(meta + 4, keySize);
+            mem.writeLong(meta + 8, keyPtr);
+            mem.writeLong(meta + 16, valPtr);
+            mem.writeLong(meta + 24, collisionPtr);
+
+            return meta;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Object key, Object val) throws IgniteCheckedException {
+            A.notNull(val, "val");
+
+            int keyHash = U.hash(key.hashCode());
+
+            // Write value.
+            long valPtr = write(12, val, valSer);
+            int valSize = writtenSize() - 12;
+
+            valueSize(valPtr, valSize);
+
+            // Find position in table.
+            int idx = keyHash & (tbl.length - 1);
+
+            long meta = tbl[idx];
+
+            // Search for our key in collisions.
+            while (meta != 0) {
+                if (keyHash(meta) == keyHash && key.equals(keyReader.readKey(meta))) { // Found key.
+                    nextValue(valPtr, value(meta));
+
+                    value(meta, valPtr);
+
+                    return;
+                }
+
+                meta = collision(meta);
+            }
+
+            // Write key.
+            long keyPtr = write(0, key, keySer);
+            int keySize = writtenSize();
+
+            nextValue(valPtr, 0);
+
+            tbl[idx] = createMeta(keyHash, keySize, keyPtr, valPtr, tbl[idx]);
+
+            if (++keys > (tbl.length >>> 2) * 3)
+                rehash();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimapBase.java
new file mode 100644
index 0000000..f62a354
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopHashMultimapBase.java
@@ -0,0 +1,208 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+
+import java.util.*;
+
+/**
+ * Base class for hash multimaps.
+ */
+public abstract class HadoopHashMultimapBase extends HadoopMultimapBase {
+    /**
+     * @param jobInfo Job info.
+     * @param mem Memory.
+     */
+    protected HadoopHashMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
+        super(jobInfo, mem);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("visit");
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+        return new Input(taskCtx);
+    }
+
+    /**
+     * @return Hash table capacity.
+     */
+    public abstract int capacity();
+
+    /**
+     * @param idx Index in hash table.
+     * @return Meta page pointer.
+     */
+    protected abstract long meta(int idx);
+
+    /**
+     * @param meta Meta pointer.
+     * @return Key hash.
+     */
+    protected int keyHash(long meta) {
+        return mem.readInt(meta);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Key size.
+     */
+    protected int keySize(long meta) {
+        return mem.readInt(meta + 4);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Key pointer.
+     */
+    protected long key(long meta) {
+        return mem.readLong(meta + 8);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Value pointer.
+     */
+    protected long value(long meta) {
+        return mem.readLong(meta + 16);
+    }
+    /**
+     * @param meta Meta pointer.
+     * @param val Value pointer.
+     */
+    protected void value(long meta, long val) {
+        mem.writeLong(meta + 16, val);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @return Collision pointer.
+     */
+    protected long collision(long meta) {
+        return mem.readLong(meta + 24);
+    }
+
+    /**
+     * @param meta Meta pointer.
+     * @param collision Collision pointer.
+     */
+    protected void collision(long meta, long collision) {
+        assert meta != collision : meta;
+
+        mem.writeLong(meta + 24, collision);
+    }
+
+    /**
+     * Reader for key and value.
+     */
+    protected class Reader extends ReaderBase {
+        /**
+         * @param ser Serialization.
+         */
+        protected Reader(GridHadoopSerialization ser) {
+            super(ser);
+        }
+
+        /**
+         * @param meta Meta pointer.
+         * @return Key.
+         */
+        public Object readKey(long meta) {
+            assert meta > 0 : meta;
+
+            try {
+                return read(key(meta), keySize(meta));
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+    }
+
+    /**
+     * Task input.
+     */
+    protected class Input implements GridHadoopTaskInput {
+        /** */
+        private int idx = -1;
+
+        /** */
+        private long metaPtr;
+
+        /** */
+        private final int cap;
+
+        /** */
+        private final Reader keyReader;
+
+        /** */
+        private final Reader valReader;
+
+        /**
+         * @param taskCtx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        public Input(GridHadoopTaskContext taskCtx) throws IgniteCheckedException {
+            cap = capacity();
+
+            keyReader = new Reader(taskCtx.keySerialization());
+            valReader = new Reader(taskCtx.valueSerialization());
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean next() {
+            if (metaPtr != 0) {
+                metaPtr = collision(metaPtr);
+
+                if (metaPtr != 0)
+                    return true;
+            }
+
+            while (++idx < cap) { // Scan table.
+                metaPtr = meta(idx);
+
+                if (metaPtr != 0)
+                    return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object key() {
+            return keyReader.readKey(metaPtr);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Iterator<?> values() {
+            return new ValueIterator(value(metaPtr), valReader);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            keyReader.close();
+            valReader.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimap.java
new file mode 100644
index 0000000..e1fa1f1
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimap.java
@@ -0,0 +1,112 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Multimap for hadoop intermediate results.
+ */
+@SuppressWarnings("PublicInnerClass")
+public interface HadoopMultimap extends AutoCloseable {
+    /**
+     * Incrementally visits all the keys and values in the map.
+     *
+     * @param ignoreLastVisited Flag indicating that visiting must be started from the beginning.
+     * @param v Visitor.
+     * @return {@code false} If visiting was impossible.
+     */
+    public boolean visit(boolean ignoreLastVisited, Visitor v) throws IgniteCheckedException;
+
+    /**
+     * @param ctx Task context.
+     * @return Adder.
+     * @throws IgniteCheckedException If failed.
+     */
+    public Adder startAdding(GridHadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param taskCtx Task context.
+     * @return Task input.
+     * @throws IgniteCheckedException If failed.
+     */
+    public GridHadoopTaskInput input(GridHadoopTaskContext taskCtx)
+        throws IgniteCheckedException;
+
+    /** {@inheritDoc} */
+    @Override public void close();
+
+    /**
+     * Adder.
+     */
+    public interface Adder extends GridHadoopTaskOutput {
+        /**
+         * @param in Data input.
+         * @param reuse Reusable key.
+         * @return Key.
+         * @throws IgniteCheckedException If failed.
+         */
+        public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException;
+    }
+
+    /**
+     * Key add values to.
+     */
+    public interface Key {
+        /**
+         * @param val Value.
+         */
+        public void add(Value val);
+    }
+
+    /**
+     * Value.
+     */
+    public interface Value {
+        /**
+         * @return Size in bytes.
+         */
+        public int size();
+
+        /**
+         * @param ptr Pointer.
+         */
+        public void copyTo(long ptr);
+    }
+
+    /**
+     * Key and values visitor.
+     */
+    public interface Visitor {
+        /**
+         * @param keyPtr Key pointer.
+         * @param keySize Key size.
+         */
+        public void onKey(long keyPtr, int keySize) throws IgniteCheckedException;
+
+        /**
+         * @param valPtr Value pointer.
+         * @param valSize Value size.
+         */
+        public void onValue(long valPtr, int valSize) throws IgniteCheckedException;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
new file mode 100644
index 0000000..4aa6e9e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopMultimapBase.java
@@ -0,0 +1,368 @@
+/*
+ * 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.shuffle.collections;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.streams.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
+
+/**
+ * Base class for all multimaps.
+ */
+public abstract class HadoopMultimapBase implements HadoopMultimap {
+    /** */
+    protected final GridUnsafeMemory mem;
+
+    /** */
+    protected final int pageSize;
+
+    /** */
+    private final Collection<GridLongList> allPages = new ConcurrentLinkedQueue<>();
+
+    /**
+     * @param jobInfo Job info.
+     * @param mem Memory.
+     */
+    protected HadoopMultimapBase(GridHadoopJobInfo jobInfo, GridUnsafeMemory mem) {
+        assert jobInfo != null;
+        assert mem != null;
+
+        this.mem = mem;
+
+        pageSize = get(jobInfo, SHUFFLE_OFFHEAP_PAGE_SIZE, 32 * 1024);
+    }
+
+    /**
+     * @param ptrs Page pointers.
+     */
+    private void deallocate(GridLongList ptrs) {
+        while (!ptrs.isEmpty())
+            mem.release(ptrs.remove(), ptrs.remove());
+    }
+
+    /**
+     * @param valPtr Value page pointer.
+     * @param nextValPtr Next value page pointer.
+     */
+    protected void nextValue(long valPtr, long nextValPtr) {
+        mem.writeLong(valPtr, nextValPtr);
+    }
+
+    /**
+     * @param valPtr Value page pointer.
+     * @return Next value page pointer.
+     */
+    protected long nextValue(long valPtr) {
+        return mem.readLong(valPtr);
+    }
+
+    /**
+     * @param valPtr Value page pointer.
+     * @param size Size.
+     */
+    protected void valueSize(long valPtr, int size) {
+        mem.writeInt(valPtr + 8, size);
+    }
+
+    /**
+     * @param valPtr Value page pointer.
+     * @return Value size.
+     */
+    protected int valueSize(long valPtr) {
+        return mem.readInt(valPtr + 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void close() {
+        for (GridLongList list : allPages)
+            deallocate(list);
+    }
+
+    /**
+     * Reader for key and value.
+     */
+    protected class ReaderBase implements AutoCloseable {
+        /** */
+        private Object tmp;
+
+        /** */
+        private final GridHadoopSerialization ser;
+
+        /** */
+        private final HadoopDataInStream in = new HadoopDataInStream(mem);
+
+        /**
+         * @param ser Serialization.
+         */
+        protected ReaderBase(GridHadoopSerialization ser) {
+            assert ser != null;
+
+            this.ser = ser;
+        }
+
+        /**
+         * @param valPtr Value page pointer.
+         * @return Value.
+         */
+        public Object readValue(long valPtr) {
+            assert valPtr > 0 : valPtr;
+
+            try {
+                return read(valPtr + 12, valueSize(valPtr));
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
+        }
+
+        /**
+         * Resets temporary object to the given one.
+         *
+         * @param tmp Temporary object for reuse.
+         */
+        public void resetReusedObject(Object tmp) {
+            this.tmp = tmp;
+        }
+
+        /**
+         * @param ptr Pointer.
+         * @param size Object size.
+         * @return Object.
+         */
+        protected Object read(long ptr, long size) throws IgniteCheckedException {
+            in.buffer().set(ptr, size);
+
+            tmp = ser.read(in, tmp);
+
+            return tmp;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            ser.close();
+        }
+    }
+
+    /**
+     * Base class for adders.
+     */
+    protected abstract class AdderBase implements Adder {
+        /** */
+        protected final GridHadoopSerialization keySer;
+
+        /** */
+        protected final GridHadoopSerialization valSer;
+
+        /** */
+        private final HadoopDataOutStream out;
+
+        /** */
+        private long writeStart;
+
+        /** Size and pointer pairs list. */
+        private final GridLongList pages = new GridLongList(16);
+
+        /**
+         * @param ctx Task context.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected AdderBase(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+            valSer = ctx.valueSerialization();
+            keySer = ctx.keySerialization();
+
+            out = new HadoopDataOutStream(mem) {
+                @Override public long move(long size) {
+                    long ptr = super.move(size);
+
+                    if (ptr == 0) // Was not able to move - not enough free space.
+                        ptr = allocateNextPage(size);
+
+                    assert ptr != 0;
+
+                    return ptr;
+                }
+            };
+        }
+
+        /**
+         * @param requestedSize Requested size.
+         * @return Next write pointer.
+         */
+        private long allocateNextPage(long requestedSize) {
+            int writtenSize = writtenSize();
+
+            long newPageSize = Math.max(writtenSize + requestedSize, pageSize);
+            long newPagePtr = mem.allocate(newPageSize);
+
+            pages.add(newPageSize);
+            pages.add(newPagePtr);
+
+            HadoopOffheapBuffer b = out.buffer();
+
+            b.set(newPagePtr, newPageSize);
+
+            if (writtenSize != 0) {
+                mem.copyMemory(writeStart, newPagePtr, writtenSize);
+
+                b.move(writtenSize);
+            }
+
+            writeStart = newPagePtr;
+
+            return b.move(requestedSize);
+        }
+
+        /**
+         * @return Fixed pointer.
+         */
+        private long fixAlignment() {
+            HadoopOffheapBuffer b = out.buffer();
+
+            long ptr = b.pointer();
+
+            if ((ptr & 7L) != 0) { // Address is not aligned by octet.
+                ptr = (ptr + 8L) & ~7L;
+
+                b.pointer(ptr);
+            }
+
+            return ptr;
+        }
+
+        /**
+         * @param off Offset.
+         * @param o Object.
+         * @return Page pointer.
+         * @throws IgniteCheckedException If failed.
+         */
+        protected long write(int off, Object o, GridHadoopSerialization ser) throws IgniteCheckedException {
+            writeStart = fixAlignment();
+
+            if (off != 0)
+                out.move(off);
+
+            ser.write(out, o);
+
+            return writeStart;
+        }
+
+        /**
+         * @param size Size.
+         * @return Pointer.
+         */
+        protected long allocate(int size) {
+            writeStart = fixAlignment();
+
+            out.move(size);
+
+            return writeStart;
+        }
+
+        /**
+         * Rewinds local allocation pointer to the given pointer if possible.
+         *
+         * @param ptr Pointer.
+         */
+        protected void localDeallocate(long ptr) {
+            HadoopOffheapBuffer b = out.buffer();
+
+            if (b.isInside(ptr))
+                b.pointer(ptr);
+            else
+                b.reset();
+        }
+
+        /**
+         * @return Written size.
+         */
+        protected int writtenSize() {
+            return (int)(out.buffer().pointer() - writeStart);
+        }
+
+        /** {@inheritDoc} */
+        @Override public Key addKey(DataInput in, @Nullable Key reuse) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws IgniteCheckedException {
+            allPages.add(pages);
+
+            keySer.close();
+            valSer.close();
+        }
+    }
+
+    /**
+     * Iterator over values.
+     */
+    protected class ValueIterator implements Iterator<Object> {
+        /** */
+        private long valPtr;
+
+        /** */
+        private final ReaderBase valReader;
+
+        /**
+         * @param valPtr Value page pointer.
+         * @param valReader Value reader.
+         */
+        protected ValueIterator(long valPtr, ReaderBase valReader) {
+            this.valPtr = valPtr;
+            this.valReader = valReader;
+        }
+
+        /**
+         * @param valPtr Head value pointer.
+         */
+        public void head(long valPtr) {
+            this.valPtr = valPtr;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean hasNext() {
+            return valPtr != 0;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object next() {
+            if (!hasNext())
+                throw new NoSuchElementException();
+
+            Object res = valReader.readValue(valPtr);
+
+            valPtr = nextValue(valPtr);
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void remove() {
+            throw new UnsupportedOperationException();
+        }
+    }
+}


[25/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopWrapper.java
deleted file mode 100644
index 5586e72..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/IgfsHadoopWrapper.java
+++ /dev/null
@@ -1,511 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.hadoop.conf.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopEndpoint.*;
-import static org.apache.ignite.internal.igfs.hadoop.IgfsHadoopUtils.*;
-
-/**
- * Wrapper for IGFS server.
- */
-public class IgfsHadoopWrapper implements IgfsHadoop {
-    /** Delegate. */
-    private final AtomicReference<Delegate> delegateRef = new AtomicReference<>();
-
-    /** Authority. */
-    private final String authority;
-
-    /** Connection string. */
-    private final IgfsHadoopEndpoint endpoint;
-
-    /** Log directory. */
-    private final String logDir;
-
-    /** Configuration. */
-    private final Configuration conf;
-
-    /** Logger. */
-    private final Log log;
-
-    /**
-     * Constructor.
-     *
-     * @param authority Authority (connection string).
-     * @param logDir Log directory for server.
-     * @param conf Configuration.
-     * @param log Current logger.
-     */
-    public IgfsHadoopWrapper(String authority, String logDir, Configuration conf, Log log) throws IOException {
-        try {
-            this.authority = authority;
-            this.endpoint = new IgfsHadoopEndpoint(authority);
-            this.logDir = logDir;
-            this.conf = conf;
-            this.log = log;
-        }
-        catch (IgniteCheckedException e) {
-            throw new IOException("Failed to parse endpoint: " + authority, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsHandshakeResponse>() {
-            @Override public IgfsHandshakeResponse apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) {
-                return hndResp;
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close(boolean force) {
-        Delegate delegate = delegateRef.get();
-
-        if (delegate != null && delegateRef.compareAndSet(delegate, null))
-            delegate.close(force);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(final IgfsPath path) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsFile>() {
-            @Override public IgfsFile apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.info(path);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsFile>() {
-            @Override public IgfsFile apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.update(path, props);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime)
-        throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Boolean>() {
-            @Override public Boolean apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.setTimes(path, accessTime, modificationTime);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean rename(final IgfsPath src, final IgfsPath dest) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Boolean>() {
-            @Override public Boolean apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.rename(src, dest);
-            }
-        }, src);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean delete(final IgfsPath path, final boolean recursive) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Boolean>() {
-            @Override public Boolean apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.delete(path, recursive);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(final IgfsPath path, final long start,
-        final long len) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Collection<IgfsBlockLocation>>() {
-            @Override public Collection<IgfsBlockLocation> apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.affinity(path, start, len);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(final IgfsPath path) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsPathSummary>() {
-            @Override public IgfsPathSummary apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.contentSummary(path);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Boolean>() {
-            @Override public Boolean apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.mkdirs(path, props);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(final IgfsPath path) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Collection<IgfsFile>>() {
-            @Override public Collection<IgfsFile> apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.listFiles(path);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(final IgfsPath path) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<Collection<IgfsPath>>() {
-            @Override public Collection<IgfsPath> apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.listPaths(path);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsStatus fsStatus() throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsStatus>() {
-            @Override public IgfsStatus apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp)
-                throws IgniteCheckedException, IOException {
-                return hadoop.fsStatus();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(final IgfsPath path) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsHadoopStreamDelegate>() {
-            @Override public IgfsHadoopStreamDelegate apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.open(path);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate open(final IgfsPath path, final int seqReadsBeforePrefetch)
-        throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsHadoopStreamDelegate>() {
-            @Override public IgfsHadoopStreamDelegate apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.open(path, seqReadsBeforePrefetch);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate create(final IgfsPath path, final boolean overwrite,
-        final boolean colocate, final int replication, final long blockSize, @Nullable final Map<String, String> props)
-        throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsHadoopStreamDelegate>() {
-            @Override public IgfsHadoopStreamDelegate apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.create(path, overwrite, colocate, replication, blockSize, props);
-            }
-        }, path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHadoopStreamDelegate append(final IgfsPath path, final boolean create,
-        @Nullable final Map<String, String> props) throws IOException {
-        return withReconnectHandling(new FileSystemClosure<IgfsHadoopStreamDelegate>() {
-            @Override public IgfsHadoopStreamDelegate apply(IgfsHadoopEx hadoop,
-                IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException {
-                return hadoop.append(path, create, props);
-            }
-        }, path);
-    }
-
-    /**
-     * Execute closure which is not path-specific.
-     *
-     * @param clo Closure.
-     * @return Result.
-     * @throws IOException If failed.
-     */
-    private <T> T withReconnectHandling(FileSystemClosure<T> clo) throws IOException {
-        return withReconnectHandling(clo, null);
-    }
-
-    /**
-     * Execute closure.
-     *
-     * @param clo Closure.
-     * @param path Path for exceptions.
-     * @return Result.
-     * @throws IOException If failed.
-     */
-    private <T> T withReconnectHandling(final FileSystemClosure<T> clo, @Nullable IgfsPath path)
-        throws IOException {
-        Exception err = null;
-
-        for (int i = 0; i < 2; i++) {
-            Delegate curDelegate = null;
-
-            boolean close = false;
-            boolean force = false;
-
-            try {
-                curDelegate = delegate();
-
-                assert curDelegate != null;
-
-                close = curDelegate.doomed;
-
-                return clo.apply(curDelegate.hadoop, curDelegate.hndResp);
-            }
-            catch (IgfsHadoopCommunicationException e) {
-                if (curDelegate != null && !curDelegate.doomed) {
-                    // Try getting rid fo faulty delegate ASAP.
-                    delegateRef.compareAndSet(curDelegate, null);
-
-                    close = true;
-                    force = true;
-                }
-
-                if (log.isDebugEnabled())
-                    log.debug("Failed to send message to a server: " + e);
-
-                err = e;
-            }
-            catch (IgniteCheckedException e) {
-                throw IgfsHadoopUtils.cast(e, path != null ? path.toString() : null);
-            }
-            finally {
-                if (close) {
-                    assert curDelegate != null;
-
-                    curDelegate.close(force);
-                }
-            }
-        }
-
-        throw new IOException("Failed to communicate with IGFS.", err);
-    }
-
-    /**
-     * Get delegate creating it if needed.
-     *
-     * @return Delegate.
-     */
-    private Delegate delegate() throws IgfsHadoopCommunicationException {
-        Exception err = null;
-
-        // 1. If delegate is set, return it immediately.
-        Delegate curDelegate = delegateRef.get();
-
-        if (curDelegate != null)
-            return curDelegate;
-
-        // 2. Guess that we are in the same VM.
-        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_EMBED, authority, false)) {
-            IgfsEx igfs = null;
-
-            if (endpoint.grid() == null) {
-                try {
-                    Ignite ignite = G.ignite();
-
-                    igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
-                }
-                catch (Exception e) {
-                    err = e;
-                }
-            }
-            else {
-                for (Ignite ignite : G.allGrids()) {
-                    try {
-                        igfs = (IgfsEx)ignite.fileSystem(endpoint.igfs());
-
-                        break;
-                    }
-                    catch (Exception e) {
-                        err = e;
-                    }
-                }
-            }
-
-            if (igfs != null) {
-                IgfsHadoopEx hadoop = null;
-
-                try {
-                    hadoop = new IgfsHadoopInProc(igfs, log);
-
-                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof IgfsHadoopCommunicationException)
-                        hadoop.close(true);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to in-proc IGFS, fallback to IPC mode.", e);
-
-                    err = e;
-                }
-            }
-        }
-
-        // 3. Try connecting using shmem.
-        if (!parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_SHMEM, authority, false)) {
-            if (curDelegate == null && !U.isWindows()) {
-                IgfsHadoopEx hadoop = null;
-
-                try {
-                    hadoop = new IgfsHadoopOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
-
-                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof IgfsHadoopCommunicationException)
-                        hadoop.close(true);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to out-proc local IGFS using shmem.", e);
-
-                    err = e;
-                }
-            }
-        }
-
-        // 4. Try local TCP connection.
-        boolean skipLocTcp = parameter(conf, PARAM_IGFS_ENDPOINT_NO_LOCAL_TCP, authority, false);
-
-        if (!skipLocTcp) {
-            if (curDelegate == null) {
-                IgfsHadoopEx hadoop = null;
-
-                try {
-                    hadoop = new IgfsHadoopOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
-                        log);
-
-                    curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (e instanceof IgfsHadoopCommunicationException)
-                        hadoop.close(true);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Failed to connect to out-proc local IGFS using TCP.", e);
-
-                    err = e;
-                }
-            }
-        }
-
-        // 5. Try remote TCP connection.
-        if (curDelegate == null && (skipLocTcp || !F.eq(LOCALHOST, endpoint.host()))) {
-            IgfsHadoopEx hadoop = null;
-
-            try {
-                hadoop = new IgfsHadoopOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
-
-                curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
-            }
-            catch (IOException | IgniteCheckedException e) {
-                if (e instanceof IgfsHadoopCommunicationException)
-                    hadoop.close(true);
-
-                if (log.isDebugEnabled())
-                    log.debug("Failed to connect to out-proc remote IGFS using TCP.", e);
-
-                err = e;
-            }
-        }
-
-        if (curDelegate != null) {
-            if (!delegateRef.compareAndSet(null, curDelegate))
-                curDelegate.doomed = true;
-
-            return curDelegate;
-        }
-        else
-            throw new IgfsHadoopCommunicationException("Failed to connect to IGFS: " + endpoint, err);
-    }
-
-    /**
-     * File system operation closure.
-     */
-    private static interface FileSystemClosure<T> {
-        /**
-         * Call closure body.
-         *
-         * @param hadoop RPC handler.
-         * @param hndResp Handshake response.
-         * @return Result.
-         * @throws IgniteCheckedException If failed.
-         * @throws IOException If failed.
-         */
-        public T apply(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp) throws IgniteCheckedException, IOException;
-    }
-
-    /**
-     * Delegate.
-     */
-    private static class Delegate {
-        /** RPC handler. */
-        private final IgfsHadoopEx hadoop;
-
-        /** Handshake request. */
-        private final IgfsHandshakeResponse hndResp;
-
-        /** Close guard. */
-        private final AtomicBoolean closeGuard = new AtomicBoolean();
-
-        /** Whether this delegate must be closed at the end of the next invocation. */
-        private boolean doomed;
-
-        /**
-         * Constructor.
-         *
-         * @param hadoop Hadoop.
-         * @param hndResp Handshake response.
-         */
-        private Delegate(IgfsHadoopEx hadoop, IgfsHandshakeResponse hndResp) {
-            this.hadoop = hadoop;
-            this.hndResp = hndResp;
-        }
-
-        /**
-         * Close underlying RPC handler.
-         *
-         * @param force Force flag.
-         */
-        private void close(boolean force) {
-            if (closeGuard.compareAndSet(false, true))
-                hadoop.close(force);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
index 01a7471..1788595 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
@@ -153,7 +153,7 @@ public class HadoopDefaultMapReducePlanner implements GridHadoopMapReducePlanner
             GridHadoopFileBlock split0 = (GridHadoopFileBlock)split;
 
             if (IGFS_SCHEME.equalsIgnoreCase(split0.file().getScheme())) {
-                IgfsHadoopEndpoint endpoint = new IgfsHadoopEndpoint(split0.file().getAuthority());
+                HadoopIgfsEndpoint endpoint = new HadoopIgfsEndpoint(split0.file().getAuthority());
 
                 IgfsEx igfs = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
new file mode 100644
index 0000000..04f2d0c
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -0,0 +1,1967 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.permission.*;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+
+/**
+ * Hadoop 2.x compliant file system.
+ */
+public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonAbstractTest {
+    /** Group size. */
+    public static final int GRP_SIZE = 128;
+
+    /** Thread count for multithreaded tests. */
+    private static final int THREAD_CNT = 8;
+
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Barrier for multithreaded tests. */
+    private static CyclicBarrier barrier;
+
+    /** File system. */
+    private static AbstractFileSystem fs;
+
+    /** Default IGFS mode. */
+    protected IgfsMode mode;
+
+    /** Primary file system URI. */
+    protected URI primaryFsUri;
+
+    /** Primary file system configuration. */
+    protected Configuration primaryFsCfg;
+
+    /**
+     * Constructor.
+     *
+     * @param mode Default IGFS mode.
+     */
+    protected HadoopIgfs20FileSystemAbstractSelfTest(IgfsMode mode) {
+        this.mode = mode;
+    }
+
+    /**
+     * Gets primary file system URI path.
+     *
+     * @return Primary file system URI path.
+     */
+    protected abstract String primaryFileSystemUriPath();
+
+    /**
+     * Gets primary file system config path.
+     *
+     * @return Primary file system config path.
+     */
+    protected abstract String primaryFileSystemConfigPath();
+
+    /**
+     * Get primary IPC endpoint configuration.
+     *
+     * @param gridName Grid name.
+     * @return IPC primary endpoint configuration.
+     */
+    protected abstract Map<String, String>  primaryIpcEndpointConfiguration(String gridName);
+
+    /**
+     * Gets secondary file system URI path.
+     *
+     * @return Secondary file system URI path.
+     */
+    protected abstract String secondaryFileSystemUriPath();
+
+    /**
+     * Gets secondary file system config path.
+     *
+     * @return Secondary file system config path.
+     */
+    protected abstract String secondaryFileSystemConfigPath();
+
+    /**
+     * Get secondary IPC endpoint configuration.
+     *
+     * @return Secondary IPC endpoint configuration.
+     */
+    protected abstract Map<String, String>  secondaryIpcEndpointConfiguration();
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startNodes();
+    }
+
+    /**
+     * Starts the nodes for this test.
+     *
+     * @throws Exception If failed.
+     */
+    private void startNodes() throws Exception {
+        if (mode != PRIMARY) {
+            // Start secondary IGFS.
+            IgfsConfiguration igfsCfg = new IgfsConfiguration();
+
+            igfsCfg.setDataCacheName("partitioned");
+            igfsCfg.setMetaCacheName("replicated");
+            igfsCfg.setName("igfs_secondary");
+            igfsCfg.setIpcEndpointConfiguration(secondaryIpcEndpointConfiguration());
+            igfsCfg.setManagementPort(-1);
+            igfsCfg.setBlockSize(512 * 1024);
+            igfsCfg.setPrefetchBlocks(1);
+
+            CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+            cacheCfg.setName("partitioned");
+            cacheCfg.setCacheMode(PARTITIONED);
+            cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+            cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(GRP_SIZE));
+            cacheCfg.setBackups(0);
+            cacheCfg.setQueryIndexEnabled(false);
+            cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+            metaCacheCfg.setName("replicated");
+            metaCacheCfg.setCacheMode(REPLICATED);
+            metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            metaCacheCfg.setQueryIndexEnabled(false);
+            metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            cfg.setGridName("grid_secondary");
+
+            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+            discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+            cfg.setDiscoverySpi(discoSpi);
+            cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
+            cfg.setIgfsConfiguration(igfsCfg);
+            cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+            cfg.setLocalHost(U.getLocalHost().getHostAddress());
+            cfg.setCommunicationSpi(communicationSpi());
+
+            G.start(cfg);
+        }
+
+        startGrids(4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getTestGridName() {
+        return "grid";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(cacheConfiguration(gridName));
+        cfg.setIgfsConfiguration(igfsConfiguration(gridName));
+        cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setCommunicationSpi(communicationSpi());
+
+        return cfg;
+    }
+
+    /**
+     * Gets cache configuration.
+     *
+     * @param gridName Grid name.
+     * @return Cache configuration.
+     */
+    protected CacheConfiguration[] cacheConfiguration(String gridName) {
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setName("partitioned");
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(GRP_SIZE));
+        cacheCfg.setBackups(0);
+        cacheCfg.setQueryIndexEnabled(false);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("replicated");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setQueryIndexEnabled(false);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        return new CacheConfiguration[] {metaCacheCfg, cacheCfg};
+    }
+
+    /**
+     * Gets IGFS configuration.
+     *
+     * @param gridName Grid name.
+     * @return IGFS configuration.
+     */
+    protected IgfsConfiguration igfsConfiguration(String gridName) throws IgniteCheckedException {
+        IgfsConfiguration cfg = new IgfsConfiguration();
+
+        cfg.setDataCacheName("partitioned");
+        cfg.setMetaCacheName("replicated");
+        cfg.setName("igfs");
+        cfg.setPrefetchBlocks(1);
+        cfg.setMaxSpaceSize(64 * 1024 * 1024);
+        cfg.setDefaultMode(mode);
+
+        if (mode != PRIMARY)
+            cfg.setSecondaryFileSystem(new IgniteHadoopSecondaryFileSystem(secondaryFileSystemUriPath(),
+                secondaryFileSystemConfigPath()));
+
+        cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
+        cfg.setManagementPort(-1);
+
+        cfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
+
+        return cfg;
+    }
+
+    /** @return Communication SPI. */
+    private CommunicationSpi communicationSpi() {
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        return commSpi;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        G.stopAll(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        primaryFsUri = new URI(primaryFileSystemUriPath());
+
+        primaryFsCfg = new Configuration();
+
+        primaryFsCfg.addResource(U.resolveIgniteUrl(primaryFileSystemConfigPath()));
+
+        fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
+
+        barrier = new CyclicBarrier(THREAD_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        try {
+            fs.delete(new Path("/"), true);
+        }
+        catch (Exception ignore) {
+            // No-op.
+        }
+
+        U.closeQuiet((Closeable)fs);
+    }
+
+    /** @throws Exception If failed. */
+    public void testStatus() throws Exception {
+
+        try (FSDataOutputStream file = fs.create(new Path("/file1"), EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()))) {
+            file.write(new byte[1024 * 1024]);
+        }
+
+        FsStatus status = fs.getFsStatus();
+
+        assertEquals(4, grid(0).cluster().nodes().size());
+
+        long used = 0, max = 0;
+
+        for (int i = 0; i < 4; i++) {
+            IgniteFs igfs = grid(i).fileSystem("igfs");
+
+            IgfsMetrics metrics = igfs.metrics();
+
+            used += metrics.localSpaceSize();
+            max += metrics.maxSpaceSize();
+        }
+
+        assertEquals(used, status.getUsed());
+        assertEquals(max, status.getCapacity());
+    }
+
+    /** @throws Exception If failed. */
+    public void testTimes() throws Exception {
+        Path file = new Path("/file1");
+
+        long now = System.currentTimeMillis();
+
+        try (FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()))) {
+            os.write(new byte[1024 * 1024]);
+        }
+
+        FileStatus status = fs.getFileStatus(file);
+
+        assertTrue(status.getAccessTime() >= now);
+        assertTrue(status.getModificationTime() >= now);
+
+        long accessTime = now - 10 * 60 * 1000;
+        long modificationTime = now - 5 * 60 * 1000;
+
+        fs.setTimes(file, modificationTime, accessTime);
+
+        status = fs.getFileStatus(file);
+        assertEquals(accessTime, status.getAccessTime());
+        assertEquals(modificationTime, status.getModificationTime());
+
+        // Check listing is updated as well.
+        FileStatus[] files = fs.listStatus(new Path("/"));
+
+        assertEquals(1, files.length);
+
+        assertEquals(file.getName(), files[0].getPath().getName());
+        assertEquals(accessTime, files[0].getAccessTime());
+        assertEquals(modificationTime, files[0].getModificationTime());
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.setTimes(new Path("/unknownFile"), 0, 0);
+
+                return null;
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testCreateCheckParameters() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.create(null, EnumSet.noneOf(CreateFlag.class),
+                    Options.CreateOpts.perms(FsPermission.getDefault()));
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testCreateBase() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
+        Path file = new Path(dir, "someFile");
+
+        assertPathDoesNotExist(fs, file);
+
+        FsPermission fsPerm = new FsPermission((short)644);
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(fsPerm));
+
+        // Try to write something in file.
+        os.write("abc".getBytes());
+
+        os.close();
+
+        // Check file status.
+        FileStatus fileStatus = fs.getFileStatus(file);
+
+        assertFalse(fileStatus.isDirectory());
+        assertEquals(file, fileStatus.getPath());
+        assertEquals(fsPerm, fileStatus.getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    public void testCreateCheckOverwrite() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
+        final Path file = new Path(dir, "someFile");
+
+        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        out.close();
+
+        // Check intermediate directory permissions.
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir).getPermission());
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir.getParent()).getPermission());
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir.getParent().getParent()).getPermission());
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.create(file, EnumSet.noneOf(CreateFlag.class),
+                    Options.CreateOpts.perms(FsPermission.getDefault()));
+            }
+        }, PathExistsException.class, null);
+
+        // Overwrite should be successful.
+        FSDataOutputStream out1 = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        out1.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteIfNoSuchPath() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
+
+        assertPathDoesNotExist(fs, dir);
+
+        assertFalse(fs.delete(dir, true));
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteSuccessfulIfPathIsOpenedToRead() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "myFile");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        final int cnt = 5 * IgfsConfiguration.DFLT_BLOCK_SIZE; // Write 5 blocks.
+
+        for (int i = 0; i < cnt; i++)
+            os.writeInt(i);
+
+        os.close();
+
+        final FSDataInputStream is = fs.open(file, -1);
+
+        for (int i = 0; i < cnt / 2; i++)
+            assertEquals(i, is.readInt());
+
+        assert fs.delete(file, false);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.getFileStatus(file);
+
+                return null;
+            }
+        }, FileNotFoundException.class, null);
+
+        is.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteIfFilePathExists() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "myFile");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        assertTrue(fs.delete(file, false));
+
+        assertPathDoesNotExist(fs, file);
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteIfDirectoryPathExists() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
+
+        FSDataOutputStream os = fs.create(dir, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        assertTrue(fs.delete(dir, false));
+
+        assertPathDoesNotExist(fs, dir);
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteFailsIfNonRecursive() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
+
+        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        final Path someDir2 = new Path(fsHome, "/someDir1/someDir2");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.delete(someDir2, false);
+
+                return null;
+            }
+        }, PathIsNotEmptyDirectoryException.class, null);
+
+        assertPathExists(fs, someDir2);
+        assertPathExists(fs, someDir3);
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteRecursively() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
+
+        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        Path someDir2 = new Path(fsHome, "/someDir1/someDir2");
+
+        assertTrue(fs.delete(someDir2, true));
+
+        assertPathDoesNotExist(fs, someDir2);
+        assertPathDoesNotExist(fs, someDir3);
+    }
+
+    /** @throws Exception If failed. */
+    public void testDeleteRecursivelyFromRoot() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
+
+        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        Path root = new Path(fsHome, "/");
+
+        assertTrue(fs.delete(root, true));
+
+        assertPathDoesNotExist(fs, someDir3);
+        assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1/someDir2"));
+        assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1"));
+        assertPathExists(fs, root);
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetPermissionCheckDefaultPermission() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        fs.setPermission(file, null);
+
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file).getPermission());
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file.getParent()).getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetPermissionCheckNonRecursiveness() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        Path tmpDir = new Path(fsHome, "/tmp");
+
+        FsPermission perm = new FsPermission((short)123);
+
+        fs.setPermission(tmpDir, perm);
+
+        assertEquals(perm, fs.getFileStatus(tmpDir).getPermission());
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file).getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    @SuppressWarnings("OctalInteger")
+    public void testSetPermission() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        for (short i = 0; i <= 0777; i += 7) {
+            FsPermission perm = new FsPermission(i);
+
+            fs.setPermission(file, perm);
+
+            assertEquals(perm, fs.getFileStatus(file).getPermission());
+        }
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetPermissionIfOutputStreamIsNotClosed() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "myFile");
+
+        FsPermission perm = new FsPermission((short)123);
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        fs.setPermission(file, perm);
+
+        os.close();
+
+        assertEquals(perm, fs.getFileStatus(file).getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwnerCheckParametersPathIsNull() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.setOwner(null, "aUser", "aGroup");
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: p");
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwnerCheckParametersUserIsNull() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.setOwner(file, null, "aGroup");
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: username");
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwnerCheckParametersGroupIsNull() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                fs.setOwner(file, "aUser", null);
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: grpName");
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwner() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        fs.setOwner(file, "aUser", "aGroup");
+
+        assertEquals("aUser", fs.getFileStatus(file).getOwner());
+        assertEquals("aGroup", fs.getFileStatus(file).getGroup());
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwnerIfOutputStreamIsNotClosed() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "myFile");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        fs.setOwner(file, "aUser", "aGroup");
+
+        os.close();
+
+        assertEquals("aUser", fs.getFileStatus(file).getOwner());
+        assertEquals("aGroup", fs.getFileStatus(file).getGroup());
+    }
+
+    /** @throws Exception If failed. */
+    public void testSetOwnerCheckNonRecursiveness() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "/tmp/my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        Path tmpDir = new Path(fsHome, "/tmp");
+
+        fs.setOwner(file, "fUser", "fGroup");
+        fs.setOwner(tmpDir, "dUser", "dGroup");
+
+        assertEquals("dUser", fs.getFileStatus(tmpDir).getOwner());
+        assertEquals("dGroup", fs.getFileStatus(tmpDir).getGroup());
+
+        assertEquals("fUser", fs.getFileStatus(file).getOwner());
+        assertEquals("fGroup", fs.getFileStatus(file).getGroup());
+    }
+
+    /** @throws Exception If failed. */
+    public void testOpenCheckParametersPathIsNull() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.open(null, 1024);
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testOpenNoSuchPath() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "someFile");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.open(file, 1024);
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testOpenIfPathIsAlreadyOpened() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "someFile");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        FSDataInputStream is1 = fs.open(file);
+        FSDataInputStream is2 = fs.open(file);
+
+        is1.close();
+        is2.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testOpen() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "someFile");
+
+        int cnt = 2 * 1024;
+
+        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        for (long i = 0; i < cnt; i++)
+            out.writeLong(i);
+
+        out.close();
+
+        FSDataInputStream in = fs.open(file, 1024);
+
+        for (long i = 0; i < cnt; i++)
+            assertEquals(i, in.readLong());
+
+        in.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testAppendIfPathPointsToDirectory() throws Exception {
+        final Path fsHome = new Path(primaryFsUri);
+        final Path dir = new Path(fsHome, "/tmp");
+        Path file = new Path(dir, "my");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.create(new Path(fsHome, dir), EnumSet.of(CreateFlag.APPEND),
+                    Options.CreateOpts.perms(FsPermission.getDefault()));
+            }
+        }, IOException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testAppendIfFileIsAlreadyBeingOpenedToWrite() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "someFile");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        FSDataOutputStream appendOs = fs.create(file, EnumSet.of(CreateFlag.APPEND),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                return fs.create(file, EnumSet.of(CreateFlag.APPEND),
+                    Options.CreateOpts.perms(FsPermission.getDefault()));
+            }
+        }, IOException.class, null);
+
+        appendOs.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testAppend() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path file = new Path(fsHome, "someFile");
+
+        int cnt = 1024;
+
+        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        for (int i = 0; i < cnt; i++)
+            out.writeLong(i);
+
+        out.close();
+
+        out = fs.create(file, EnumSet.of(CreateFlag.APPEND),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        for (int i = cnt; i < cnt * 2; i++)
+            out.writeLong(i);
+
+        out.close();
+
+        FSDataInputStream in = fs.open(file, 1024);
+
+        for (int i = 0; i < cnt * 2; i++)
+            assertEquals(i, in.readLong());
+
+        in.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameCheckParametersSrcPathIsNull() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "someFile");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.rename(null, file);
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameCheckParametersDstPathIsNull() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path file = new Path(fsHome, "someFile");
+
+        fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault())).close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override
+            public Object call() throws Exception {
+                fs.rename(file, null);
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameIfSrcPathDoesNotExist() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path srcFile = new Path(fsHome, "srcFile");
+        final Path dstFile = new Path(fsHome, "dstFile");
+
+        assertPathDoesNotExist(fs, srcFile);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.rename(srcFile, dstFile);
+
+                return null;
+            }
+        }, FileNotFoundException.class, null);
+
+        assertPathDoesNotExist(fs, dstFile);
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameIfSrcPathIsAlreadyBeingOpenedToWrite() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path srcFile = new Path(fsHome, "srcFile");
+        Path dstFile = new Path(fsHome, "dstFile");
+
+        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        os = fs.create(srcFile, EnumSet.of(CreateFlag.APPEND),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        fs.rename(srcFile, dstFile);
+
+        assertPathExists(fs, dstFile);
+
+        String testStr = "Test";
+
+        try {
+            os.writeBytes(testStr);
+        }
+        finally {
+            os.close();
+        }
+
+        try (FSDataInputStream is = fs.open(dstFile)) {
+            byte[] buf = new byte[testStr.getBytes().length];
+
+            is.readFully(buf);
+
+            assertEquals(testStr, new String(buf));
+        }
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameFileIfDstPathExists() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        final Path srcFile = new Path(fsHome, "srcFile");
+        final Path dstFile = new Path(fsHome, "dstFile");
+
+        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        os = fs.create(dstFile, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.rename(srcFile, dstFile);
+
+                return null;
+            }
+        }, FileAlreadyExistsException.class, null);
+
+        assertPathExists(fs, srcFile);
+        assertPathExists(fs, dstFile);
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameFile() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path srcFile = new Path(fsHome, "/tmp/srcFile");
+        Path dstFile = new Path(fsHome, "/tmp/dstFile");
+
+        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        fs.rename(srcFile, dstFile);
+
+        assertPathDoesNotExist(fs, srcFile);
+        assertPathExists(fs, dstFile);
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameIfSrcPathIsAlreadyBeingOpenedToRead() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path srcFile = new Path(fsHome, "srcFile");
+        Path dstFile = new Path(fsHome, "dstFile");
+
+        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        int cnt = 1024;
+
+        for (int i = 0; i < cnt; i++)
+            os.writeInt(i);
+
+        os.close();
+
+        FSDataInputStream is = fs.open(srcFile);
+
+        for (int i = 0; i < cnt; i++) {
+            if (i == 100)
+                // Rename file during the read process.
+                fs.rename(srcFile, dstFile);
+
+            assertEquals(i, is.readInt());
+        }
+
+        assertPathDoesNotExist(fs, srcFile);
+        assertPathExists(fs, dstFile);
+
+        os.close();
+        is.close();
+    }
+
+    /** @throws Exception If failed. */
+    public void _testRenameDirectoryIfDstPathExists() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path srcDir = new Path(fsHome, "/tmp/");
+        Path dstDir = new Path(fsHome, "/tmpNew/");
+
+        FSDataOutputStream os = fs.create(new Path(srcDir, "file1"), EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        os = fs.create(new Path(dstDir, "file2"), EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        fs.rename(srcDir, dstDir);
+
+        assertPathExists(fs, dstDir);
+        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp"));
+        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp/file1"));
+    }
+
+    /** @throws Exception If failed. */
+    public void testRenameDirectory() throws Exception {
+        Path fsHome = new Path(primaryFsUri);
+        Path dir = new Path(fsHome, "/tmp/");
+        Path newDir = new Path(fsHome, "/tmpNew/");
+
+        FSDataOutputStream os = fs.create(new Path(dir, "myFile"), EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        os.close();
+
+        fs.rename(dir, newDir);
+
+        assertPathDoesNotExist(fs, dir);
+        assertPathExists(fs, newDir);
+    }
+
+    /** @throws Exception If failed. */
+    public void testListStatusIfPathIsNull() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.listStatus(null);
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testListStatusIfPathDoesNotExist() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.listStatus(new Path("/someDir"));
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /**
+     * Test directory listing.
+     *
+     * @throws Exception If failed.
+     */
+    public void testListStatus() throws Exception {
+        Path igfsHome = new Path(primaryFsUri);
+
+        // Test listing of an empty directory.
+        Path dir = new Path(igfsHome, "dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        FileStatus[] list = fs.listStatus(dir);
+
+        assert list.length == 0;
+
+        // Test listing of a not empty directory.
+        Path subDir = new Path(dir, "subDir");
+
+        fs.mkdir(subDir, FsPermission.getDefault(), true);
+
+        Path file = new Path(dir, "file");
+
+        FSDataOutputStream fos = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        fos.close();
+
+        list = fs.listStatus(dir);
+
+        assert list.length == 2;
+
+        String listRes1 = list[0].getPath().getName();
+        String listRes2 = list[1].getPath().getName();
+
+        assert "subDir".equals(listRes1) && "file".equals(listRes2) || "subDir".equals(listRes2) &&
+            "file".equals(listRes1);
+
+        // Test listing of a file.
+        list = fs.listStatus(file);
+
+        assert list.length == 1;
+
+        assert "file".equals(list[0].getPath().getName());
+    }
+
+    /** @throws Exception If failed. */
+    public void testMkdirsIfPathIsNull() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.mkdir(null, FsPermission.getDefault(), true);
+
+                return null;
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testMkdirsIfPermissionIsNull() throws Exception {
+        Path dir = new Path("/tmp");
+
+        fs.mkdir(dir, null, true);
+
+        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir).getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    @SuppressWarnings("OctalInteger")
+    public void testMkdirs() throws Exception {
+        Path fsHome = new Path(primaryFileSystemUriPath());
+        Path dir = new Path(fsHome, "/tmp/staging");
+        Path nestedDir = new Path(dir, "nested");
+
+        FsPermission dirPerm = FsPermission.createImmutable((short)0700);
+        FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
+
+        fs.mkdir(dir, dirPerm, true);
+        fs.mkdir(nestedDir, nestedDirPerm, true);
+
+        assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
+        assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
+    }
+
+    /** @throws Exception If failed. */
+    public void testGetFileStatusIfPathIsNull() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.getFileStatus(null);
+            }
+        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
+    }
+
+    /** @throws Exception If failed. */
+    public void testGetFileStatusIfPathDoesNotExist() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.getFileStatus(new Path("someDir"));
+            }
+        }, FileNotFoundException.class, "File not found: someDir");
+    }
+
+    /** @throws Exception If failed. */
+    public void testGetFileBlockLocationsIfFileStatusIsNull() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                // Argument is checked by Hadoop.
+                return fs.getFileBlockLocations(null, 1, 2);
+            }
+        }, NullPointerException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testGetFileBlockLocationsIfFileStatusReferenceNotExistingPath() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.getFileBlockLocations(new Path("/someFile"), 1, 2);
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testGetFileBlockLocations() throws Exception {
+        Path igfsHome = new Path(primaryFsUri);
+
+        Path file = new Path(igfsHome, "someFile");
+
+        try (OutputStream out = new BufferedOutputStream(fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault())))) {
+            byte[] data = new byte[128 * 1024];
+
+            for (int i = 0; i < 100; i++)
+                out.write(data);
+
+            out.flush();
+        }
+
+        try (FSDataInputStream in = fs.open(file, 1024 * 1024)) {
+            byte[] data = new byte[128 * 1024];
+
+            int read;
+
+            do {
+                read = in.read(data);
+            }
+            while (read > 0);
+        }
+
+        FileStatus status = fs.getFileStatus(file);
+
+        int grpLen = 128 * 512 * 1024;
+
+        int grpCnt = (int)((status.getLen() + grpLen - 1) / grpLen);
+
+        BlockLocation[] locations = fs.getFileBlockLocations(file, 0, status.getLen());
+
+        assertEquals(grpCnt, locations.length);
+    }
+
+    /** @throws Exception If failed. */
+    public void testZeroReplicationFactor() throws Exception {
+        // This test doesn't make sense for any mode except of PRIMARY.
+        if (mode == PRIMARY) {
+            Path igfsHome = new Path(primaryFsUri);
+
+            Path file = new Path(igfsHome, "someFile");
+
+            try (FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+                Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.repFac((short)1))) {
+                out.write(new byte[1024 * 1024]);
+            }
+
+            IgniteFs igfs = grid(0).fileSystem("igfs");
+
+            IgfsPath filePath = new IgfsPath("/someFile");
+
+            IgfsFile fileInfo = igfs.info(filePath);
+
+            Collection<IgfsBlockLocation> locations = igfs.affinity(filePath, 0, fileInfo.length());
+
+            assertEquals(1, locations.size());
+
+            IgfsBlockLocation location = F.first(locations);
+
+            assertEquals(1, location.nodeIds().size());
+        }
+    }
+
+    /**
+     * Ensure that when running in multithreaded mode only one create() operation succeed.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedCreate() throws Exception {
+        Path dir = new Path(new Path(primaryFsUri), "/dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        final Path file = new Path(dir, "file");
+
+        fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault())).close();
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        final Collection<Integer> errs = new GridConcurrentHashSet<>(THREAD_CNT, 1.0f, THREAD_CNT);
+
+        multithreaded(new Runnable() {
+            @Override public void run() {
+                int idx = cnt.getAndIncrement();
+
+                byte[] data = new byte[256];
+
+                Arrays.fill(data, (byte)idx);
+
+                FSDataOutputStream os = null;
+
+                try {
+                    os = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
+                        Options.CreateOpts.perms(FsPermission.getDefault()));
+
+                    os.write(data);
+                }
+                catch (IOException ignore) {
+                    errs.add(idx);
+                }
+                finally {
+                    U.awaitQuiet(barrier);
+
+                    U.closeQuiet(os);
+                }
+            }
+        }, THREAD_CNT);
+
+        // Only one thread could obtain write lock on the file.
+        assert errs.size() == THREAD_CNT - 1 : "Invalid errors count [expected=" + (THREAD_CNT - 1) + ", actual=" +
+            errs.size() + ']';
+
+        int idx = -1;
+
+        for (int i = 0; i < THREAD_CNT; i++) {
+            if (!errs.remove(i)) {
+                idx = i;
+
+                break;
+            }
+        }
+
+        byte[] expData = new byte[256];
+
+        Arrays.fill(expData, (byte)idx);
+
+        FSDataInputStream is = fs.open(file);
+
+        byte[] data = new byte[256];
+
+        is.read(data);
+
+        is.close();
+
+        assert Arrays.equals(expData, data);
+    }
+
+    /**
+     * Ensure that when running in multithreaded mode only one append() operation succeed.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedAppend() throws Exception {
+        Path dir = new Path(new Path(primaryFsUri), "/dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        final Path file = new Path(dir, "file");
+
+        fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault())).close();
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        final Collection<Integer> errs = new GridConcurrentHashSet<>(THREAD_CNT, 1.0f, THREAD_CNT);
+
+        multithreaded(new Runnable() {
+            @Override public void run() {
+                int idx = cnt.getAndIncrement();
+
+                byte[] data = new byte[256];
+
+                Arrays.fill(data, (byte)idx);
+
+                U.awaitQuiet(barrier);
+
+                FSDataOutputStream os = null;
+
+                try {
+                    os = fs.create(file, EnumSet.of(CreateFlag.APPEND),
+                        Options.CreateOpts.perms(FsPermission.getDefault()));
+
+                    os.write(data);
+                }
+                catch (IOException ignore) {
+                    errs.add(idx);
+                }
+                finally {
+                    U.awaitQuiet(barrier);
+
+                    U.closeQuiet(os);
+                }
+            }
+        }, THREAD_CNT);
+
+        // Only one thread could obtain write lock on the file.
+        assert errs.size() == THREAD_CNT - 1;
+
+        int idx = -1;
+
+        for (int i = 0; i < THREAD_CNT; i++) {
+            if (!errs.remove(i)) {
+                idx = i;
+
+                break;
+            }
+        }
+
+        byte[] expData = new byte[256];
+
+        Arrays.fill(expData, (byte)idx);
+
+        FSDataInputStream is = fs.open(file);
+
+        byte[] data = new byte[256];
+
+        is.read(data);
+
+        is.close();
+
+        assert Arrays.equals(expData, data);
+    }
+
+    /**
+     * Test concurrent reads within the file.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedOpen() throws Exception {
+        final byte[] dataChunk = new byte[256];
+
+        for (int i = 0; i < dataChunk.length; i++)
+            dataChunk[i] = (byte)i;
+
+        Path dir = new Path(new Path(primaryFsUri), "/dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        final Path file = new Path(dir, "file");
+
+        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()));
+
+        // Write 256 * 2048 = 512Kb of data.
+        for (int i = 0; i < 2048; i++)
+            os.write(dataChunk);
+
+        os.close();
+
+        final AtomicBoolean err = new AtomicBoolean();
+
+        multithreaded(new Runnable() {
+            @Override
+            public void run() {
+                FSDataInputStream is = null;
+
+                try {
+                    int pos = ThreadLocalRandom8.current().nextInt(2048);
+
+                    try {
+                        is = fs.open(file);
+                    }
+                    finally {
+                        U.awaitQuiet(barrier);
+                    }
+
+                    is.seek(256 * pos);
+
+                    byte[] buf = new byte[256];
+
+                    for (int i = pos; i < 2048; i++) {
+                        // First perform normal read.
+                        int read = is.read(buf);
+
+                        assert read == 256;
+
+                        Arrays.equals(dataChunk, buf);
+                    }
+
+                    int res = is.read(buf);
+
+                    assert res == -1;
+                }
+                catch (IOException ignore) {
+                    err.set(true);
+                }
+                finally {
+                    U.closeQuiet(is);
+                }
+            }
+        }, THREAD_CNT);
+
+        assert !err.get();
+    }
+
+    /**
+     * Test concurrent creation of multiple directories.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultithreadedMkdirs() throws Exception {
+        final Path dir = new Path(new Path("igfs:///"), "/dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        final int depth = 3;
+        final int entryCnt = 5;
+
+        final AtomicBoolean err = new AtomicBoolean();
+
+        multithreaded(new Runnable() {
+            @Override public void run() {
+                Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
+
+                queue.add(F.t(0, dir));
+
+                U.awaitQuiet(barrier);
+
+                while (!queue.isEmpty()) {
+                    IgniteBiTuple<Integer, Path> t = queue.pollFirst();
+
+                    int curDepth = t.getKey();
+                    Path curPath = t.getValue();
+
+                    if (curDepth <= depth) {
+                        int newDepth = curDepth + 1;
+
+                        // Create directories.
+                        for (int i = 0; i < entryCnt; i++) {
+                            Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
+
+                            try {
+                                fs.mkdir(subDir, FsPermission.getDefault(), true);
+                            }
+                            catch (IOException ignore) {
+                                err.set(true);
+                            }
+
+                            queue.addLast(F.t(newDepth, subDir));
+                        }
+                    }
+                }
+            }
+        }, THREAD_CNT);
+
+        // Ensure there were no errors.
+        assert !err.get();
+
+        // Ensure correct folders structure.
+        Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
+
+        queue.add(F.t(0, dir));
+
+        while (!queue.isEmpty()) {
+            IgniteBiTuple<Integer, Path> t = queue.pollFirst();
+
+            int curDepth = t.getKey();
+            Path curPath = t.getValue();
+
+            if (curDepth <= depth) {
+                int newDepth = curDepth + 1;
+
+                // Create directories.
+                for (int i = 0; i < entryCnt; i++) {
+                    Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
+
+                    assertNotNull(fs.getFileStatus(subDir));
+
+                    queue.add(F.t(newDepth, subDir));
+                }
+            }
+        }
+    }
+
+    /**
+     * Test concurrent deletion of the same directory with advanced structure.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testMultithreadedDelete() throws Exception {
+        final Path dir = new Path(new Path(primaryFsUri), "/dir");
+
+        fs.mkdir(dir, FsPermission.getDefault(), true);
+
+        int depth = 3;
+        int entryCnt = 5;
+
+        Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
+
+        queue.add(F.t(0, dir));
+
+        while (!queue.isEmpty()) {
+            IgniteBiTuple<Integer, Path> t = queue.pollFirst();
+
+            int curDepth = t.getKey();
+            Path curPath = t.getValue();
+
+            if (curDepth < depth) {
+                int newDepth = curDepth + 1;
+
+                // Create directories.
+                for (int i = 0; i < entryCnt; i++) {
+                    Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
+
+                    fs.mkdir(subDir, FsPermission.getDefault(), true);
+
+                    queue.addLast(F.t(newDepth, subDir));
+                }
+            }
+            else {
+                // Create files.
+                for (int i = 0; i < entryCnt; i++) {
+                    Path file = new Path(curPath, "file " + i);
+
+                    fs.create(file, EnumSet.noneOf(CreateFlag.class),
+                        Options.CreateOpts.perms(FsPermission.getDefault())).close();
+                }
+            }
+        }
+
+        final AtomicBoolean err = new AtomicBoolean();
+
+        multithreaded(new Runnable() {
+            @Override public void run() {
+                try {
+                    U.awaitQuiet(barrier);
+
+                    fs.delete(dir, true);
+                }
+                catch (FileNotFoundException ignore) {
+                    // No-op.
+                }
+                catch (IOException ignore) {
+                    err.set(true);
+                }
+            }
+        }, THREAD_CNT);
+
+        // Ensure there were no errors.
+        assert !err.get();
+
+        // Ensure the directory was actually deleted.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                fs.getFileStatus(dir);
+
+                return null;
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /** @throws Exception If failed. */
+    public void testConsistency() throws Exception {
+        // Default buffers values
+        checkConsistency(-1, 1, -1, -1, 1, -1);
+        checkConsistency(-1, 10, -1, -1, 10, -1);
+        checkConsistency(-1, 100, -1, -1, 100, -1);
+        checkConsistency(-1, 1000, -1, -1, 1000, -1);
+        checkConsistency(-1, 10000, -1, -1, 10000, -1);
+        checkConsistency(-1, 100000, -1, -1, 100000, -1);
+
+        checkConsistency(65 * 1024 + 13, 100000, -1, -1, 100000, -1);
+
+        checkConsistency(-1, 100000, 2 * 4 * 1024 + 17, -1, 100000, -1);
+
+        checkConsistency(-1, 100000, -1, 65 * 1024 + 13, 100000, -1);
+
+        checkConsistency(-1, 100000, -1, -1, 100000, 2 * 4 * 1024 + 17);
+
+        checkConsistency(65 * 1024 + 13, 100000, 2 * 4 * 1024 + 13, 65 * 1024 + 149, 100000, 2 * 4 * 1024 + 157);
+    }
+
+    /**
+     * Verifies that client reconnects after connection to the server has been lost.
+     *
+     * @throws Exception If error occurs.
+     */
+    public void testClientReconnect() throws Exception {
+        final Path igfsHome = new Path(primaryFsUri);
+
+        final Path filePath = new Path(igfsHome, "someFile");
+
+        final FSDataOutputStream s = fs.create(filePath, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault())); // Open stream before stopping IGFS.
+
+        try {
+            G.stopAll(true); // Stop the server.
+
+            startNodes(); // Start server again.
+
+            // Check that client is again operational.
+            fs.mkdir(new Path("igfs:///dir1/dir2"), FsPermission.getDefault(), true);
+
+            // However, the streams, opened before disconnect, should not be valid.
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Nullable @Override public Object call() throws Exception {
+                    s.write("test".getBytes());
+
+                    s.flush();
+
+                    return null;
+                }
+            }, IOException.class, null);
+
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    fs.getFileStatus(filePath);
+
+                    return null;
+                }
+            }, FileNotFoundException.class, null);
+        }
+        finally {
+            U.closeQuiet(s);
+        }
+    }
+
+    /**
+     * Verifies that client reconnects after connection to the server has been lost (multithreaded mode).
+     *
+     * @throws Exception If error occurs.
+     */
+    public void testClientReconnectMultithreaded() throws Exception {
+        final ConcurrentLinkedQueue<FileSystem> q = new ConcurrentLinkedQueue<>();
+
+        Configuration cfg = new Configuration();
+
+        for (Map.Entry<String, String> entry : primaryFsCfg)
+            cfg.set(entry.getKey(), entry.getValue());
+
+        cfg.setBoolean("fs.igfs.impl.disable.cache", true);
+
+        final int nClients = 16;
+
+        // Initialize clients.
+        for (int i = 0; i < nClients; i++)
+            q.add(FileSystem.get(primaryFsUri, cfg));
+
+        G.stopAll(true); // Stop the server.
+
+        startNodes(); // Start server again.
+
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                FileSystem fs = q.poll();
+
+                try {
+                    // Check that client is again operational.
+                    assertTrue(fs.mkdirs(new Path("igfs:///" + Thread.currentThread().getName())));
+
+                    return true;
+                }
+                finally {
+                    U.closeQuiet(fs);
+                }
+            }
+        }, nClients, "test-client");
+    }
+
+    /**
+     * Checks consistency of create --> open --> append --> open operations with different buffer sizes.
+     *
+     * @param createBufSize Buffer size used for file creation.
+     * @param writeCntsInCreate Count of times to write in file creation.
+     * @param openAfterCreateBufSize Buffer size used for file opening after creation.
+     * @param appendBufSize Buffer size used for file appending.
+     * @param writeCntsInAppend Count of times to write in file appending.
+     * @param openAfterAppendBufSize Buffer size used for file opening after appending.
+     * @throws Exception If failed.
+     */
+    private void checkConsistency(int createBufSize, int writeCntsInCreate, int openAfterCreateBufSize,
+        int appendBufSize, int writeCntsInAppend, int openAfterAppendBufSize) throws Exception {
+        final Path igfsHome = new Path(primaryFsUri);
+
+        Path file = new Path(igfsHome, "/someDir/someInnerDir/someFile");
+
+        if (createBufSize == -1)
+            createBufSize = fs.getServerDefaults().getFileBufferSize();
+
+        if (appendBufSize == -1)
+            appendBufSize = fs.getServerDefaults().getFileBufferSize();
+
+        FSDataOutputStream os = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
+            Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.bufferSize(createBufSize));
+
+        for (int i = 0; i < writeCntsInCreate; i++)
+            os.writeInt(i);
+
+        os.close();
+
+        FSDataInputStream is = fs.open(file, openAfterCreateBufSize);
+
+        for (int i = 0; i < writeCntsInCreate; i++)
+            assertEquals(i, is.readInt());
+
+        is.close();
+
+        os = fs.create(file, EnumSet.of(CreateFlag.APPEND),
+            Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.bufferSize(appendBufSize));
+
+        for (int i = writeCntsInCreate; i < writeCntsInCreate + writeCntsInAppend; i++)
+            os.writeInt(i);
+
+        os.close();
+
+        is = fs.open(file, openAfterAppendBufSize);
+
+        for (int i = 0; i < writeCntsInCreate + writeCntsInAppend; i++)
+            assertEquals(i, is.readInt());
+
+        is.close();
+    }
+
+    /**
+     * Test expected failures for 'close' operation.
+     *
+     * @param fs File system to test.
+     * @param msg Expected exception message.
+     */
+    public void assertCloseFails(final FileSystem fs, String msg) {
+        GridTestUtils.assertThrows(log, new Callable() {
+            @Override public Object call() throws Exception {
+                fs.close();
+
+                return null;
+            }
+        }, IOException.class, msg);
+    }
+
+    /**
+     * Test expected failures for 'get content summary' operation.
+     *
+     * @param fs File system to test.
+     * @param path Path to evaluate content summary for.
+     */
+    private void assertContentSummaryFails(final FileSystem fs, final Path path) {
+        GridTestUtils.assertThrows(log, new Callable<ContentSummary>() {
+            @Override public ContentSummary call() throws Exception {
+                return fs.getContentSummary(path);
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /**
+     * Assert that a given path exists in a given FileSystem.
+     *
+     * @param fs FileSystem to check.
+     * @param p Path to check.
+     * @throws IOException if the path does not exist.
+     */
+    private void assertPathExists(AbstractFileSystem fs, Path p) throws IOException {
+        FileStatus fileStatus = fs.getFileStatus(p);
+
+        assertEquals(p, fileStatus.getPath());
+        assertNotSame(0, fileStatus.getModificationTime());
+    }
+
+    /**
+     * Check path does not exist in a given FileSystem.
+     *
+     * @param fs FileSystem to check.
+     * @param path Path to check.
+     */
+    private void assertPathDoesNotExist(final AbstractFileSystem fs, final Path path) {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fs.getFileStatus(path);
+            }
+        }, FileNotFoundException.class, null);
+    }
+
+    /** Helper class to encapsulate source and destination folders. */
+    @SuppressWarnings({"PublicInnerClass", "PublicField"})
+    public static final class Config {
+        /** Source file system. */
+        public final AbstractFileSystem srcFs;
+
+        /** Source path to work with. */
+        public final Path src;
+
+        /** Destination file system. */
+        public final AbstractFileSystem destFs;
+
+        /** Destination path to work with. */
+        public final Path dest;
+
+        /**
+         * Copying task configuration.
+         *
+         * @param srcFs Source file system.
+         * @param src Source path.
+         * @param destFs Destination file system.
+         * @param dest Destination path.
+         */
+        public Config(AbstractFileSystem srcFs, Path src, AbstractFileSystem destFs, Path dest) {
+            this.srcFs = srcFs;
+            this.src = src;
+            this.destFs = destFs;
+            this.dest = dest;
+        }
+    }
+
+    /**
+     * Convert path for exception message testing purposes.
+     *
+     * @param path Path.
+     * @return Converted path.
+     * @throws Exception If failed.
+     */
+    private Path convertPath(Path path) throws Exception {
+        if (mode != PROXY)
+            return path;
+        else {
+            URI secondaryUri = new URI(secondaryFileSystemUriPath());
+
+            URI pathUri = path.toUri();
+
+            return new Path(new URI(pathUri.getScheme() != null ? secondaryUri.getScheme() : null,
+                pathUri.getAuthority() != null ? secondaryUri.getAuthority() : null, pathUri.getPath(), null, null));
+        }
+    }
+}


[30/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (5).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
deleted file mode 100644
index 4de4d53..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopIpcIgfsIo.java
+++ /dev/null
@@ -1,599 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.ipc.*;
-import org.apache.ignite.internal.util.ipc.shmem.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.concurrent.locks.*;
-
-/**
- * IO layer implementation based on blocking IPC streams.
- */
-@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-public class HadoopIpcIgfsIo implements HadoopIgfsIo {
-    /** Logger. */
-    private Log log;
-
-    /** Request futures map. */
-    private ConcurrentMap<Long, HadoopIgfsFuture> reqMap =
-        new ConcurrentHashMap8<>();
-
-    /** Request ID counter. */
-    private AtomicLong reqIdCnt = new AtomicLong();
-
-    /** Endpoint. */
-    private IpcEndpoint endpoint;
-
-    /** Endpoint output stream. */
-    private IgfsDataOutputStream out;
-
-    /** Protocol. */
-    private final IgfsMarshaller marsh;
-
-    /** Client reader thread. */
-    private Thread reader;
-
-    /** Lock for graceful shutdown. */
-    private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
-
-    /** Stopping flag. */
-    private volatile boolean stopping;
-
-    /** Server endpoint address. */
-    private final String endpointAddr;
-
-    /** Number of open file system sessions. */
-    private final AtomicInteger activeCnt = new AtomicInteger(1);
-
-    /** Event listeners. */
-    private final Collection<HadoopIgfsIpcIoListener> lsnrs =
-        new GridConcurrentHashSet<>();
-
-    /** Cached connections. */
-    private static final ConcurrentMap<String, HadoopIpcIgfsIo> ipcCache =
-        new ConcurrentHashMap8<>();
-
-    /** Striped lock that prevents multiple instance creation in {@link #get(Log, String)}. */
-    private static final GridStripedLock initLock = new GridStripedLock(32);
-
-    /**
-     * @param endpointAddr Endpoint.
-     * @param marsh Protocol.
-     * @param log Logger to use.
-     */
-    public HadoopIpcIgfsIo(String endpointAddr, IgfsMarshaller marsh, Log log) {
-        assert endpointAddr != null;
-        assert marsh != null;
-
-        this.endpointAddr = endpointAddr;
-        this.marsh = marsh;
-        this.log = log;
-    }
-
-    /**
-     * Returns a started and valid instance of this class
-     * for a given endpoint.
-     *
-     * @param log Logger to use for new instance.
-     * @param endpoint Endpoint string.
-     * @return New or existing cached instance, which is started and operational.
-     * @throws IOException If new instance was created but failed to start.
-     */
-    public static HadoopIpcIgfsIo get(Log log, String endpoint) throws IOException {
-        while (true) {
-            HadoopIpcIgfsIo clientIo = ipcCache.get(endpoint);
-
-            if (clientIo != null) {
-                if (clientIo.acquire())
-                    return clientIo;
-                else
-                    // If concurrent close.
-                    ipcCache.remove(endpoint, clientIo);
-            }
-            else {
-                Lock lock = initLock.getLock(endpoint);
-
-                lock.lock();
-
-                try {
-                    clientIo = ipcCache.get(endpoint);
-
-                    if (clientIo != null) { // Perform double check.
-                        if (clientIo.acquire())
-                            return clientIo;
-                        else
-                            // If concurrent close.
-                            ipcCache.remove(endpoint, clientIo);
-                    }
-
-                    // Otherwise try creating a new one.
-                    clientIo = new HadoopIpcIgfsIo(endpoint, new IgfsMarshaller(), log);
-
-                    try {
-                        clientIo.start();
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IOException(e.getMessage(), e);
-                    }
-
-                    HadoopIpcIgfsIo old = ipcCache.putIfAbsent(endpoint, clientIo);
-
-                    // Put in exclusive lock.
-                    assert old == null;
-
-                    return clientIo;
-                }
-                finally {
-                    lock.unlock();
-                }
-            }
-        }
-    }
-
-    /**
-     * Increases usage count for this instance.
-     *
-     * @return {@code true} if usage count is greater than zero.
-     */
-    private boolean acquire() {
-        while (true) {
-            int cnt = activeCnt.get();
-
-            if (cnt == 0) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO not acquired (count was 0): " + this);
-
-                return false;
-            }
-
-            // Need to make sure that no-one decremented count in between.
-            if (activeCnt.compareAndSet(cnt, cnt + 1)) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO acquired: " + this);
-
-                return true;
-            }
-        }
-    }
-
-    /**
-     * Releases this instance, decrementing usage count.
-     * <p>
-     * If usage count becomes zero, the instance is stopped
-     * and removed from cache.
-     */
-    public void release() {
-        while (true) {
-            int cnt = activeCnt.get();
-
-            if (cnt == 0) {
-                if (log.isDebugEnabled())
-                    log.debug("IPC IO not released (count was 0): " + this);
-
-                return;
-            }
-
-            if (activeCnt.compareAndSet(cnt, cnt - 1)) {
-                if (cnt == 1) {
-                    ipcCache.remove(endpointAddr, this);
-
-                    if (log.isDebugEnabled())
-                        log.debug("IPC IO stopping as unused: " + this);
-
-                    stop();
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("IPC IO released: " + this);
-
-                return;
-            }
-        }
-    }
-
-    /**
-     * Closes this IO instance, removing it from cache.
-     */
-    public void forceClose() {
-        if (ipcCache.remove(endpointAddr, this))
-            stop();
-    }
-
-    /**
-     * Starts the IO.
-     *
-     * @throws IgniteCheckedException If failed to connect the endpoint.
-     */
-    private void start() throws IgniteCheckedException {
-        boolean success = false;
-
-        try {
-            endpoint = IpcEndpointFactory.connectEndpoint(
-                    endpointAddr, new GridLoggerProxy(new HadoopIgfsJclLogger(log), null, null, ""));
-
-            out = new IgfsDataOutputStream(new BufferedOutputStream(endpoint.outputStream()));
-
-            reader = new ReaderThread();
-
-            // Required for Hadoop 2.x
-            reader.setDaemon(true);
-
-            reader.start();
-
-            success = true;
-        }
-        catch (IgniteCheckedException e) {
-            IpcOutOfSystemResourcesException resEx = e.getCause(IpcOutOfSystemResourcesException.class);
-
-            if (resEx != null)
-                throw new IgniteCheckedException(IpcSharedMemoryServerEndpoint.OUT_OF_RESOURCES_MSG, resEx);
-
-            throw e;
-        }
-        finally {
-            if (!success)
-                stop();
-        }
-    }
-
-    /**
-     * Shuts down the IO. No send requests will be accepted anymore, all pending futures will be failed.
-     * Close listeners will be invoked as if connection is closed by server.
-     */
-    private void stop() {
-        close0(null);
-
-        if (reader != null) {
-            try {
-                U.interrupt(reader);
-                U.join(reader);
-
-                reader = null;
-            }
-            catch (IgniteInterruptedCheckedException ignored) {
-                Thread.currentThread().interrupt();
-
-                log.warn("Got interrupted while waiting for reader thread to shut down (will return).");
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addEventListener(HadoopIgfsIpcIoListener lsnr) {
-        if (!busyLock.readLock().tryLock()) {
-            lsnr.onClose();
-
-            return;
-        }
-
-        boolean invokeNow = false;
-
-        try {
-            invokeNow = stopping;
-
-            if (!invokeNow)
-                lsnrs.add(lsnr);
-        }
-        finally {
-            busyLock.readLock().unlock();
-
-            if (invokeNow)
-                lsnr.onClose();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeEventListener(HadoopIgfsIpcIoListener lsnr) {
-        lsnrs.remove(lsnr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridPlainFuture<IgfsMessage> send(IgfsMessage msg) throws IgniteCheckedException {
-        return send(msg, null, 0, 0);
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> GridPlainFuture<T> send(IgfsMessage msg, @Nullable byte[] outBuf, int outOff,
-        int outLen) throws IgniteCheckedException {
-        assert outBuf == null || msg.command() == IgfsIpcCommand.READ_BLOCK;
-
-        if (!busyLock.readLock().tryLock())
-            throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
-                "closed).");
-
-        try {
-            if (stopping)
-                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently " +
-                    "closed).");
-
-            long reqId = reqIdCnt.getAndIncrement();
-
-            HadoopIgfsFuture<T> fut = new HadoopIgfsFuture<>();
-
-            fut.outputBuffer(outBuf);
-            fut.outputOffset(outOff);
-            fut.outputLength(outLen);
-            fut.read(msg.command() == IgfsIpcCommand.READ_BLOCK);
-
-            HadoopIgfsFuture oldFut = reqMap.putIfAbsent(reqId, fut);
-
-            assert oldFut == null;
-
-            if (log.isDebugEnabled())
-                log.debug("Sending IGFS message [reqId=" + reqId + ", msg=" + msg + ']');
-
-            byte[] hdr = IgfsMarshaller.createHeader(reqId, msg.command());
-
-            IgniteCheckedException err = null;
-
-            try {
-                synchronized (this) {
-                    marsh.marshall(msg, hdr, out);
-
-                    out.flush(); // Blocking operation + sometimes system call.
-                }
-            }
-            catch (IgniteCheckedException e) {
-                err = e;
-            }
-            catch (IOException e) {
-                err = new HadoopIgfsCommunicationException(e);
-            }
-
-            if (err != null) {
-                reqMap.remove(reqId, fut);
-
-                fut.onDone(err);
-            }
-
-            return fut;
-        }
-        finally {
-            busyLock.readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendPlain(IgfsMessage msg) throws IgniteCheckedException {
-        if (!busyLock.readLock().tryLock())
-            throw new HadoopIgfsCommunicationException("Failed to send message (client is being " +
-                "concurrently closed).");
-
-        try {
-            if (stopping)
-                throw new HadoopIgfsCommunicationException("Failed to send message (client is being concurrently closed).");
-
-            assert msg.command() == IgfsIpcCommand.WRITE_BLOCK;
-
-            IgfsStreamControlRequest req = (IgfsStreamControlRequest)msg;
-
-            byte[] hdr = IgfsMarshaller.createHeader(-1, IgfsIpcCommand.WRITE_BLOCK);
-
-            U.longToBytes(req.streamId(), hdr, 12);
-            U.intToBytes(req.length(), hdr, 20);
-
-            synchronized (this) {
-                out.write(hdr);
-                out.write(req.data(), (int)req.position(), req.length());
-
-                out.flush();
-            }
-        }
-        catch (IOException e) {
-            throw new HadoopIgfsCommunicationException(e);
-        }
-        finally {
-            busyLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * Closes client but does not wait.
-     *
-     * @param err Error.
-     */
-    private void close0(@Nullable Throwable err) {
-        busyLock.writeLock().lock();
-
-        try {
-            if (stopping)
-                return;
-
-            stopping = true;
-        }
-        finally {
-            busyLock.writeLock().unlock();
-        }
-
-        if (err == null)
-            err = new IgniteCheckedException("Failed to perform request (connection was concurrently closed before response " +
-                "is received).");
-
-        // Clean up resources.
-        U.closeQuiet(out);
-
-        if (endpoint != null)
-            endpoint.close();
-
-        // Unwind futures. We can safely iterate here because no more futures will be added.
-        Iterator<HadoopIgfsFuture> it = reqMap.values().iterator();
-
-        while (it.hasNext()) {
-            HadoopIgfsFuture fut = it.next();
-
-            fut.onDone(err);
-
-            it.remove();
-        }
-
-        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
-            lsnr.onClose();
-    }
-
-    /**
-     * Do not extend {@code GridThread} to minimize class dependencies.
-     */
-    private class ReaderThread extends Thread {
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public void run() {
-            // Error to fail pending futures.
-            Throwable err = null;
-
-            try {
-                InputStream in = endpoint.inputStream();
-
-                IgfsDataInputStream dis = new IgfsDataInputStream(in);
-
-                byte[] hdr = new byte[IgfsMarshaller.HEADER_SIZE];
-                byte[] msgHdr = new byte[IgfsControlResponse.RES_HEADER_SIZE];
-
-                while (!Thread.currentThread().isInterrupted()) {
-                    dis.readFully(hdr);
-
-                    long reqId = U.bytesToLong(hdr, 0);
-
-                    // We don't wait for write responses, therefore reqId is -1.
-                    if (reqId == -1) {
-                        // We received a response which normally should not be sent. It must contain an error.
-                        dis.readFully(msgHdr);
-
-                        assert msgHdr[4] != 0;
-
-                        String errMsg = dis.readUTF();
-
-                        // Error code.
-                        dis.readInt();
-
-                        long streamId = dis.readLong();
-
-                        for (HadoopIgfsIpcIoListener lsnr : lsnrs)
-                            lsnr.onError(streamId, errMsg);
-                    }
-                    else {
-                        HadoopIgfsFuture<Object> fut = reqMap.remove(reqId);
-
-                        if (fut == null) {
-                            String msg = "Failed to read response from server: response closure is unavailable for " +
-                                "requestId (will close connection):" + reqId;
-
-                            log.warn(msg);
-
-                            err = new IgniteCheckedException(msg);
-
-                            break;
-                        }
-                        else {
-                            try {
-                                IgfsIpcCommand cmd = IgfsIpcCommand.valueOf(U.bytesToInt(hdr, 8));
-
-                                if (log.isDebugEnabled())
-                                    log.debug("Received IGFS response [reqId=" + reqId + ", cmd=" + cmd + ']');
-
-                                Object res = null;
-
-                                if (fut.read()) {
-                                    dis.readFully(msgHdr);
-
-                                    boolean hasErr = msgHdr[4] != 0;
-
-                                    if (hasErr) {
-                                        String errMsg = dis.readUTF();
-
-                                        // Error code.
-                                        Integer errCode = dis.readInt();
-
-                                        IgfsControlResponse.throwError(errCode, errMsg);
-                                    }
-
-                                    int blockLen = U.bytesToInt(msgHdr, 5);
-
-                                    int readLen = Math.min(blockLen, fut.outputLength());
-
-                                    if (readLen > 0) {
-                                        assert fut.outputBuffer() != null;
-
-                                        dis.readFully(fut.outputBuffer(), fut.outputOffset(), readLen);
-                                    }
-
-                                    if (readLen != blockLen) {
-                                        byte[] buf = new byte[blockLen - readLen];
-
-                                        dis.readFully(buf);
-
-                                        res = buf;
-                                    }
-                                }
-                                else
-                                    res = marsh.unmarshall(cmd, hdr, dis);
-
-                                fut.onDone(res);
-                            }
-                            catch (IgniteCheckedException e) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to apply response closure (will fail request future): " +
-                                        e.getMessage());
-
-                                fut.onDone(e);
-
-                                err = e;
-                            }
-                        }
-                    }
-                }
-            }
-            catch (EOFException ignored) {
-                err = new IgniteCheckedException("Failed to read response from server (connection was closed by remote peer).");
-            }
-            catch (IOException e) {
-                if (!stopping)
-                    log.error("Failed to read data (connection will be closed)", e);
-
-                err = new HadoopIgfsCommunicationException(e);
-            }
-            catch (IgniteCheckedException e) {
-                if (!stopping)
-                    log.error("Failed to obtain endpoint input stream (connection will be closed)", e);
-
-                err = e;
-            }
-            finally {
-                close0(err);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return getClass().getSimpleName() + " [endpointAddr=" + endpointAddr + ", activeCnt=" + activeCnt +
-            ", stopping=" + stopping + ']';
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
deleted file mode 100644
index 19205c0..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutProcIgfs.java
+++ /dev/null
@@ -1,466 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.util.*;
-
-import static org.apache.ignite.internal.igfs.common.IgfsIpcCommand.*;
-
-/**
- * Communication with external process (TCP or shmem).
- */
-public class HadoopOutProcIgfs implements HadoopIgfsEx, HadoopIgfsIpcIoListener {
-    /** Expected result is boolean. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Boolean> BOOL_RES = createClosure();
-
-    /** Expected result is boolean. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, Long> LONG_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsFile> FILE_RES = createClosure();
-
-    /** Expected result is {@code IgfsHandshakeResponse} */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        IgfsHandshakeResponse> HANDSHAKE_RES = createClosure();
-
-    /** Expected result is {@code IgfsStatus} */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsStatus> STATUS_RES =
-        createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        IgfsInputStreamDescriptor> STREAM_DESCRIPTOR_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsFile>> FILE_COL_RES = createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsPath>> PATH_COL_RES = createClosure();
-
-    /** Expected result is {@code IgfsPathSummary}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>, IgfsPathSummary> SUMMARY_RES =
-        createClosure();
-
-    /** Expected result is {@code IgfsFile}. */
-    private static final GridPlainClosure<GridPlainFuture<IgfsMessage>,
-        Collection<IgfsBlockLocation>> BLOCK_LOCATION_COL_RES = createClosure();
-
-    /** Grid name. */
-    private final String grid;
-
-    /** IGFS name. */
-    private final String igfs;
-
-    /** Client log. */
-    private final Log log;
-
-    /** Client IO. */
-    private final HadoopIpcIgfsIo io;
-
-    /** Event listeners. */
-    private final Map<Long, HadoopIgfsStreamEventListener> lsnrs = new ConcurrentHashMap8<>();
-
-    /**
-     * Constructor for TCP endpoint.
-     *
-     * @param host Host.
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    public HadoopOutProcIgfs(String host, int port, String grid, String igfs, Log log) throws IOException {
-        this(host, port, grid, igfs, false, log);
-    }
-
-    /**
-     * Constructor for shmem endpoint.
-     *
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    public HadoopOutProcIgfs(int port, String grid, String igfs, Log log) throws IOException {
-        this(null, port, grid, igfs, true, log);
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param host Host.
-     * @param port Port.
-     * @param grid Grid name.
-     * @param igfs IGFS name.
-     * @param shmem Shared memory flag.
-     * @param log Client logger.
-     * @throws IOException If failed.
-     */
-    private HadoopOutProcIgfs(String host, int port, String grid, String igfs, boolean shmem, Log log)
-        throws IOException {
-        assert host != null && !shmem || host == null && shmem :
-            "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
-
-        String endpoint = host != null ? host + ":" + port : "shmem:" + port;
-
-        this.grid = grid;
-        this.igfs = igfs;
-        this.log = log;
-
-        io = HadoopIpcIgfsIo.get(log, endpoint);
-
-        io.addEventListener(this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) throws IgniteCheckedException {
-        final IgfsHandshakeRequest req = new IgfsHandshakeRequest();
-
-        req.gridName(grid);
-        req.igfsName(igfs);
-        req.logDirectory(logDir);
-
-        return io.send(req).chain(HANDSHAKE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close(boolean force) {
-        assert io != null;
-
-        io.removeEventListener(this);
-
-        if (force)
-            io.forceClose();
-        else
-            io.release();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(INFO);
-        msg.path(path);
-
-        return io.send(msg).chain(FILE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(UPDATE);
-        msg.path(path);
-        msg.properties(props);
-
-        return io.send(msg).chain(FILE_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(SET_TIMES);
-        msg.path(path);
-        msg.accessTime(accessTime);
-        msg.modificationTime(modificationTime);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(RENAME);
-        msg.path(src);
-        msg.destinationPath(dest);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(DELETE);
-        msg.path(path);
-        msg.flag(recursive);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
-        throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(AFFINITY);
-        msg.path(path);
-        msg.start(start);
-        msg.length(len);
-
-        return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(PATH_SUMMARY);
-        msg.path(path);
-
-        return io.send(msg).chain(SUMMARY_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(MAKE_DIRECTORIES);
-        msg.path(path);
-        msg.properties(props);
-
-        return io.send(msg).chain(BOOL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(LIST_FILES);
-        msg.path(path);
-
-        return io.send(msg).chain(FILE_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(LIST_PATHS);
-        msg.path(path);
-
-        return io.send(msg).chain(PATH_COL_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
-        return io.send(new IgfsStatusRequest()).chain(STATUS_RES).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_READ);
-        msg.path(path);
-        msg.flag(false);
-
-        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
-
-        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
-    }
-
-    /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path,
-        int seqReadsBeforePrefetch) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_READ);
-        msg.path(path);
-        msg.flag(true);
-        msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
-
-        IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
-
-        return new HadoopIgfsStreamDelegate(this, rmtDesc.streamId(), rmtDesc.length());
-    }
-
-    /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_CREATE);
-        msg.path(path);
-        msg.flag(overwrite);
-        msg.colocate(colocate);
-        msg.properties(props);
-        msg.replication(replication);
-        msg.blockSize(blockSize);
-
-        Long streamId = io.send(msg).chain(LONG_RES).get();
-
-        return new HadoopIgfsStreamDelegate(this, streamId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
-        final IgfsPathControlRequest msg = new IgfsPathControlRequest();
-
-        msg.command(OPEN_APPEND);
-        msg.path(path);
-        msg.flag(create);
-        msg.properties(props);
-
-        Long streamId = io.send(msg).chain(LONG_RES).get();
-
-        return new HadoopIgfsStreamDelegate(this, streamId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridPlainFuture<byte[]> readData(HadoopIgfsStreamDelegate desc, long pos, int len,
-        final @Nullable byte[] outBuf, final int outOff, final int outLen) {
-        assert len > 0;
-
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(READ_BLOCK);
-        msg.streamId((long) desc.target());
-        msg.position(pos);
-        msg.length(len);
-
-        try {
-            return io.send(msg, outBuf, outOff, outLen);
-        }
-        catch (IgniteCheckedException e) {
-            return new GridPlainFutureAdapter<>(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeData(HadoopIgfsStreamDelegate desc, byte[] data, int off, int len)
-        throws IOException {
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(WRITE_BLOCK);
-        msg.streamId((long) desc.target());
-        msg.data(data);
-        msg.position(off);
-        msg.length(len);
-
-        try {
-            io.sendPlain(msg);
-        }
-        catch (IgniteCheckedException e) {
-            throw HadoopIgfsUtils.cast(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush(HadoopIgfsStreamDelegate delegate) throws IOException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void closeStream(HadoopIgfsStreamDelegate desc) throws IOException {
-        final IgfsStreamControlRequest msg = new IgfsStreamControlRequest();
-
-        msg.command(CLOSE);
-        msg.streamId((long)desc.target());
-
-        try {
-            io.send(msg).chain(BOOL_RES).get();
-        }
-        catch (IgniteCheckedException e) {
-            throw HadoopIgfsUtils.cast(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addEventListener(HadoopIgfsStreamDelegate desc,
-        HadoopIgfsStreamEventListener lsnr) {
-        long streamId = desc.target();
-
-        HadoopIgfsStreamEventListener lsnr0 = lsnrs.put(streamId, lsnr);
-
-        assert lsnr0 == null || lsnr0 == lsnr;
-
-        if (log.isDebugEnabled())
-            log.debug("Added stream event listener [streamId=" + streamId + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeEventListener(HadoopIgfsStreamDelegate desc) {
-        long streamId = desc.target();
-
-        HadoopIgfsStreamEventListener lsnr0 = lsnrs.remove(streamId);
-
-        if (lsnr0 != null && log.isDebugEnabled())
-            log.debug("Removed stream event listener [streamId=" + streamId + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() {
-        for (HadoopIgfsStreamEventListener lsnr : lsnrs.values()) {
-            try {
-                lsnr.onClose();
-            }
-            catch (IgniteCheckedException e) {
-                log.warn("Got exception from stream event listener (will ignore): " + lsnr, e);
-            }
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(long streamId, String errMsg) {
-        HadoopIgfsStreamEventListener lsnr = lsnrs.get(streamId);
-
-        if (lsnr != null)
-            lsnr.onError(errMsg);
-        else
-            log.warn("Received write error response for not registered output stream (will ignore) " +
-                "[streamId= " + streamId + ']');
-    }
-
-    /**
-     * Creates conversion closure for given type.
-     *
-     * @param <T> Type of expected result.
-     * @return Conversion closure.
-     */
-    @SuppressWarnings("unchecked")
-    private static <T> GridPlainClosure<GridPlainFuture<IgfsMessage>, T> createClosure() {
-        return new GridPlainClosure<GridPlainFuture<IgfsMessage>, T>() {
-            @Override public T apply(GridPlainFuture<IgfsMessage> fut) throws IgniteCheckedException {
-                IgfsControlResponse res = (IgfsControlResponse)fut.get();
-
-                if (res.hasError())
-                    res.throwError();
-
-                return (T)res.response();
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
deleted file mode 100644
index ab5fa68..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/HadoopOutputIgfsStream.java
+++ /dev/null
@@ -1,201 +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.igfs.hadoop;
-
-import org.apache.commons.logging.*;
-import org.apache.ignite.*;
-import org.apache.ignite.internal.igfs.common.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * IGFS Hadoop output stream implementation.
- */
-public class HadoopOutputIgfsStream extends OutputStream implements HadoopIgfsStreamEventListener {
-    /** Log instance. */
-    private Log log;
-
-    /** Client logger. */
-    private IgfsLogger clientLog;
-
-    /** Log stream ID. */
-    private long logStreamId;
-
-    /** Server stream delegate. */
-    private HadoopIgfsStreamDelegate delegate;
-
-    /** Closed flag. */
-    private volatile boolean closed;
-
-    /** Flag set if stream was closed due to connection breakage. */
-    private boolean connBroken;
-
-    /** Error message. */
-    private volatile String errMsg;
-
-    /** Read time. */
-    private long writeTime;
-
-    /** User time. */
-    private long userTime;
-
-    /** Last timestamp. */
-    private long lastTs;
-
-    /** Amount of written bytes. */
-    private long total;
-
-    /**
-     * Creates light output stream.
-     *
-     * @param delegate Server stream delegate.
-     * @param log Logger to use.
-     * @param clientLog Client logger.
-     */
-    public HadoopOutputIgfsStream(HadoopIgfsStreamDelegate delegate, Log log,
-        IgfsLogger clientLog, long logStreamId) {
-        this.delegate = delegate;
-        this.log = log;
-        this.clientLog = clientLog;
-        this.logStreamId = logStreamId;
-
-        lastTs = System.nanoTime();
-
-        delegate.hadoop().addEventListener(delegate, this);
-    }
-
-    /**
-     * Read start.
-     */
-    private void writeStart() {
-        long now = System.nanoTime();
-
-        userTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /**
-     * Read end.
-     */
-    private void writeEnd() {
-        long now = System.nanoTime();
-
-        writeTime += now - lastTs;
-
-        lastTs = now;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
-        check();
-
-        writeStart();
-
-        try {
-            delegate.hadoop().writeData(delegate, b, off, len);
-
-            total += len;
-        }
-        finally {
-            writeEnd();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void write(int b) throws IOException {
-        write(new byte[] {(byte)b});
-
-        total++;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void flush() throws IOException {
-        delegate.hadoop().flush(delegate);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        if (!closed) {
-            if (log.isDebugEnabled())
-                log.debug("Closing output stream: " + delegate);
-
-            writeStart();
-
-            delegate.hadoop().closeStream(delegate);
-
-            markClosed(false);
-
-            writeEnd();
-
-            if (clientLog.isLogEnabled())
-                clientLog.logCloseOut(logStreamId, userTime, writeTime, total);
-
-            if (log.isDebugEnabled())
-                log.debug("Closed output stream [delegate=" + delegate + ", writeTime=" + writeTime / 1000 +
-                    ", userTime=" + userTime / 1000 + ']');
-        }
-        else if(connBroken)
-            throw new IOException(
-                "Failed to close stream, because connection was broken (data could have been lost).");
-    }
-
-    /**
-     * Marks stream as closed.
-     *
-     * @param connBroken {@code True} if connection with server was lost.
-     */
-    private void markClosed(boolean connBroken) {
-        // It is ok to have race here.
-        if (!closed) {
-            closed = true;
-
-            delegate.hadoop().removeEventListener(delegate);
-
-            this.connBroken = connBroken;
-        }
-    }
-
-    /**
-     * @throws IOException If check failed.
-     */
-    private void check() throws IOException {
-        String errMsg0 = errMsg;
-
-        if (errMsg0 != null)
-            throw new IOException(errMsg0);
-
-        if (closed) {
-            if (connBroken)
-                throw new IOException("Server connection was lost.");
-            else
-                throw new IOException("Stream is closed.");
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onClose() throws IgniteCheckedException {
-        markClosed(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onError(String errMsg) {
-        this.errMsg = errMsg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/package.html b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/package.html
deleted file mode 100644
index ec380f2..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/hadoop/package.html
+++ /dev/null
@@ -1,24 +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.
--->
-
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<html>
-<body>
-    <!-- Package description. -->
-    Contains IGFS client classes.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1ffc10f/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/package.html
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/package.html b/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/package.html
deleted file mode 100644
index 4b070d3..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/igfs/package.html
+++ /dev/null
@@ -1,24 +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.
--->
-
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<html>
-<body>
-    <!-- Package description. -->
-    Contains IGFS client and common classes.
-</body>
-</html>


[22/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (3).

Posted by vo...@apache.org.
# IGNITE-386: WIP on internal namings (3).


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

Branch: refs/heads/ignite-386
Commit: 1c4b00d4131fef7e05a661516b2b2f8fd480bb2e
Parents: 288709a
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Tue Mar 3 16:01:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Tue Mar 3 16:01:21 2015 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    |    4 +-
 .../processors/hadoop/HadoopCounters.java       |    4 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    2 +-
 .../internal/processors/hadoop/HadoopUtils.java |    6 +-
 .../taskexecutor/GridHadoopRunnableTask.java    |  268 ----
 .../HadoopEmbeddedTaskExecutor.java             |   16 +-
 .../taskexecutor/HadoopExecutorService.java     |    4 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |  268 ++++
 .../external/HadoopExternalTaskExecutor.java    |    8 +-
 .../child/GridHadoopChildProcessRunner.java     |  440 ------
 .../child/GridHadoopExternalProcessStarter.java |  296 ----
 .../child/HadoopChildProcessRunner.java         |  440 ++++++
 .../child/HadoopExternalProcessStarter.java     |  296 ++++
 .../GridHadoopAbstractCommunicationClient.java  |   96 --
 .../GridHadoopCommunicationClient.java          |   72 -
 .../GridHadoopExternalCommunication.java        | 1431 ------------------
 .../GridHadoopHandshakeTimeoutException.java    |   42 -
 .../GridHadoopIpcToNioAdapter.java              |  239 ---
 .../GridHadoopMarshallerFilter.java             |   84 -
 .../GridHadoopMessageListener.java              |   39 -
 .../GridHadoopTcpNioCommunicationClient.java    |   99 --
 .../HadoopAbstractCommunicationClient.java      |   96 ++
 .../HadoopCommunicationClient.java              |   72 +
 .../HadoopExternalCommunication.java            | 1431 ++++++++++++++++++
 .../HadoopHandshakeTimeoutException.java        |   42 +
 .../communication/HadoopIpcToNioAdapter.java    |  239 +++
 .../communication/HadoopMarshallerFilter.java   |   84 +
 .../communication/HadoopMessageListener.java    |   39 +
 .../HadoopTcpNioCommunicationClient.java        |   99 ++
 .../hadoop/v1/GridHadoopV1CleanupTask.java      |   62 -
 .../hadoop/v1/GridHadoopV1Counter.java          |  105 --
 .../hadoop/v1/GridHadoopV1MapTask.java          |  111 --
 .../hadoop/v1/GridHadoopV1OutputCollector.java  |  130 --
 .../hadoop/v1/GridHadoopV1Partitioner.java      |   44 -
 .../hadoop/v1/GridHadoopV1ReduceTask.java       |   92 --
 .../hadoop/v1/GridHadoopV1Reporter.java         |   79 -
 .../hadoop/v1/GridHadoopV1SetupTask.java        |   56 -
 .../hadoop/v1/GridHadoopV1Splitter.java         |   97 --
 .../processors/hadoop/v1/GridHadoopV1Task.java  |   95 --
 .../hadoop/v1/HadoopV1CleanupTask.java          |   62 +
 .../processors/hadoop/v1/HadoopV1Counter.java   |  105 ++
 .../processors/hadoop/v1/HadoopV1MapTask.java   |  111 ++
 .../hadoop/v1/HadoopV1OutputCollector.java      |  130 ++
 .../hadoop/v1/HadoopV1Partitioner.java          |   44 +
 .../hadoop/v1/HadoopV1ReduceTask.java           |   92 ++
 .../processors/hadoop/v1/HadoopV1Reporter.java  |   79 +
 .../processors/hadoop/v1/HadoopV1SetupTask.java |   56 +
 .../processors/hadoop/v1/HadoopV1Splitter.java  |   97 ++
 .../processors/hadoop/v1/HadoopV1Task.java      |   95 ++
 .../hadoop/v2/GridHadoopExternalSplit.java      |   87 --
 .../hadoop/v2/GridHadoopNativeCodeLoader.java   |   74 -
 .../v2/GridHadoopSerializationWrapper.java      |  133 --
 .../v2/GridHadoopShutdownHookManager.java       |   96 --
 .../hadoop/v2/GridHadoopSplitWrapper.java       |  118 --
 .../hadoop/v2/GridHadoopV2CleanupTask.java      |   73 -
 .../hadoop/v2/GridHadoopV2Context.java          |  230 ---
 .../hadoop/v2/GridHadoopV2Counter.java          |   87 --
 .../processors/hadoop/v2/GridHadoopV2Job.java   |  280 ----
 .../v2/GridHadoopV2JobResourceManager.java      |  305 ----
 .../hadoop/v2/GridHadoopV2MapTask.java          |  109 --
 .../hadoop/v2/GridHadoopV2Partitioner.java      |   44 -
 .../hadoop/v2/GridHadoopV2ReduceTask.java       |   88 --
 .../hadoop/v2/GridHadoopV2SetupTask.java        |   66 -
 .../hadoop/v2/GridHadoopV2Splitter.java         |  105 --
 .../processors/hadoop/v2/GridHadoopV2Task.java  |  181 ---
 .../hadoop/v2/GridHadoopV2TaskContext.java      |  443 ------
 .../v2/GridHadoopWritableSerialization.java     |   74 -
 .../hadoop/v2/HadoopExternalSplit.java          |   87 ++
 .../hadoop/v2/HadoopNativeCodeLoader.java       |   74 +
 .../hadoop/v2/HadoopSerializationWrapper.java   |  133 ++
 .../hadoop/v2/HadoopShutdownHookManager.java    |   96 ++
 .../hadoop/v2/HadoopSplitWrapper.java           |  118 ++
 .../hadoop/v2/HadoopV2CleanupTask.java          |   73 +
 .../processors/hadoop/v2/HadoopV2Context.java   |  230 +++
 .../processors/hadoop/v2/HadoopV2Counter.java   |   87 ++
 .../processors/hadoop/v2/HadoopV2Job.java       |  280 ++++
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  305 ++++
 .../processors/hadoop/v2/HadoopV2MapTask.java   |  109 ++
 .../hadoop/v2/HadoopV2Partitioner.java          |   44 +
 .../hadoop/v2/HadoopV2ReduceTask.java           |   88 ++
 .../processors/hadoop/v2/HadoopV2SetupTask.java |   66 +
 .../processors/hadoop/v2/HadoopV2Splitter.java  |  105 ++
 .../processors/hadoop/v2/HadoopV2Task.java      |  181 +++
 .../hadoop/v2/HadoopV2TaskContext.java          |  443 ++++++
 .../hadoop/v2/HadoopWritableSerialization.java  |   74 +
 .../GridHadoopSerializationWrapperSelfTest.java |   74 -
 .../hadoop/GridHadoopSplitWrapperSelfTest.java  |   68 -
 .../hadoop/GridHadoopTasksAllVersionsTest.java  |   30 +-
 .../hadoop/GridHadoopTasksV1Test.java           |    4 +-
 .../hadoop/GridHadoopTasksV2Test.java           |    4 +-
 .../hadoop/GridHadoopTestTaskContext.java       |  219 ---
 .../hadoop/GridHadoopV2JobSelfTest.java         |   88 --
 .../HadoopSerializationWrapperSelfTest.java     |   74 +
 .../hadoop/HadoopSplitWrapperSelfTest.java      |   68 +
 .../hadoop/HadoopTestTaskContext.java           |  219 +++
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   88 ++
 .../collections/GridHadoopAbstractMapTest.java  |    4 +-
 ...GridHadoopExternalCommunicationSelfTest.java |  209 ---
 .../HadoopExternalCommunicationSelfTest.java    |  209 +++
 .../testsuites/IgniteHadoopTestSuite.java       |    8 +-
 100 files changed, 7275 insertions(+), 7275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 6915d17..2f484d8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -93,9 +93,9 @@ public class HadoopClassLoader extends URLClassLoader {
         try {
             if (isHadoop(name)) { // Always load Hadoop classes explicitly, since Hadoop can be available in App classpath.
                 if (name.endsWith(".util.ShutdownHookManager"))  // Dirty hack to get rid of Hadoop shutdown hooks.
-                    return loadFromBytes(name, GridHadoopShutdownHookManager.class.getName());
+                    return loadFromBytes(name, HadoopShutdownHookManager.class.getName());
                 else if (name.endsWith(".util.NativeCodeLoader"))
-                    return loadFromBytes(name, GridHadoopNativeCodeLoader.class.getName());
+                    return loadFromBytes(name, HadoopNativeCodeLoader.class.getName());
 
                 return loadClassExplicitly(name, resolve);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
index ad699ec..3482640 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCounters.java
@@ -186,7 +186,7 @@ public class HadoopCounters extends Counters {
 
         for (HadoopLongCounter counter : cntrs.values()) {
             if (grpName.equals(counter.group()))
-                grpCounters.add(new GridHadoopV2Counter(counter));
+                grpCounters.add(new HadoopV2Counter(counter));
         }
 
         return grpCounters.iterator();
@@ -211,6 +211,6 @@ public class HadoopCounters extends Counters {
             cntrs.put(key, new HadoopLongCounter(grpName,cntrName));
         }
 
-        return internalCntr == null ? null : new GridHadoopV2Counter(internalCntr);
+        return internalCntr == null ? null : new HadoopV2Counter(internalCntr);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index 370b82d..2f44778 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -91,7 +91,7 @@ public class HadoopDefaultJobInfo implements GridHadoopJobInfo, Externalizable {
                     if ((jobCls0 = jobCls) == null) {
                         HadoopClassLoader ldr = new HadoopClassLoader(null);
 
-                        jobCls = jobCls0 = ldr.loadClass(GridHadoopV2Job.class.getName());
+                        jobCls = jobCls0 = ldr.loadClass(HadoopV2Job.class.getName());
                     }
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
index 46594ce..62b5a98 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
@@ -65,7 +65,7 @@ public class HadoopUtils {
      * @param hosts Hosts.
      * @throws IOException If failed.
      */
-    public static GridHadoopSplitWrapper wrapSplit(int id, Object split, String[] hosts) throws IOException {
+    public static HadoopSplitWrapper wrapSplit(int id, Object split, String[] hosts) throws IOException {
         ByteArrayOutputStream arr = new ByteArrayOutputStream();
         ObjectOutput out = new ObjectOutputStream(arr);
 
@@ -75,7 +75,7 @@ public class HadoopUtils {
 
         out.flush();
 
-        return new GridHadoopSplitWrapper(id, split.getClass().getName(), arr.toByteArray(), hosts);
+        return new HadoopSplitWrapper(id, split.getClass().getName(), arr.toByteArray(), hosts);
     }
 
     /**
@@ -84,7 +84,7 @@ public class HadoopUtils {
      * @param o Wrapper.
      * @return Split.
      */
-    public static Object unwrapSplit(GridHadoopSplitWrapper o) {
+    public static Object unwrapSplit(HadoopSplitWrapper o) {
         try {
             Writable w = (Writable)HadoopUtils.class.getClassLoader().loadClass(o.className()).newInstance();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
deleted file mode 100644
index 1ce7d4a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopRunnableTask.java
+++ /dev/null
@@ -1,268 +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.taskexecutor;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.counter.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.collections.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
-
-/**
- * Runnable task.
- */
-public abstract class GridHadoopRunnableTask implements Callable<Void> {
-    /** */
-    private final GridUnsafeMemory mem;
-
-    /** */
-    private final IgniteLogger log;
-
-    /** */
-    private final GridHadoopJob job;
-
-    /** Task to run. */
-    private final GridHadoopTaskInfo info;
-
-    /** Submit time. */
-    private final long submitTs = U.currentTimeMillis();
-
-    /** Execution start timestamp. */
-    private long execStartTs;
-
-    /** Execution end timestamp. */
-    private long execEndTs;
-
-    /** */
-    private HadoopMultimap combinerInput;
-
-    /** */
-    private volatile GridHadoopTaskContext ctx;
-
-    /** Set if task is to cancelling. */
-    private volatile boolean cancelled;
-
-    /** Node id. */
-    private UUID nodeId;
-
-    /**
-     * @param log Log.
-     * @param job Job.
-     * @param mem Memory.
-     * @param info Task info.
-     * @param nodeId Node id.
-     */
-    protected GridHadoopRunnableTask(IgniteLogger log, GridHadoopJob job, GridUnsafeMemory mem, GridHadoopTaskInfo info,
-        UUID nodeId) {
-        this.nodeId = nodeId;
-        this.log = log.getLogger(GridHadoopRunnableTask.class);
-        this.job = job;
-        this.mem = mem;
-        this.info = info;
-    }
-
-    /**
-     * @return Wait time.
-     */
-    public long waitTime() {
-        return execStartTs - submitTs;
-    }
-
-    /**
-     * @return Execution time.
-     */
-    public long executionTime() {
-        return execEndTs - execStartTs;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Void call() throws IgniteCheckedException {
-        execStartTs = U.currentTimeMillis();
-
-        Throwable err = null;
-
-        HadoopTaskState state = HadoopTaskState.COMPLETED;
-
-        HadoopPerformanceCounter perfCntr = null;
-
-        try {
-            ctx = job.getTaskContext(info);
-
-            perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
-
-            perfCntr.onTaskSubmit(info, submitTs);
-            perfCntr.onTaskPrepare(info, execStartTs);
-
-            ctx.prepareTaskEnvironment();
-
-            runTask(perfCntr);
-
-            if (info.type() == MAP && job.info().hasCombiner()) {
-                ctx.taskInfo(new GridHadoopTaskInfo(COMBINE, info.jobId(), info.taskNumber(), info.attempt(), null));
-
-                try {
-                    runTask(perfCntr);
-                }
-                finally {
-                    ctx.taskInfo(info);
-                }
-            }
-        }
-        catch (HadoopTaskCancelledException ignored) {
-            state = HadoopTaskState.CANCELED;
-        }
-        catch (Throwable e) {
-            state = HadoopTaskState.FAILED;
-            err = e;
-
-            U.error(log, "Task execution failed.", e);
-        }
-        finally {
-            execEndTs = U.currentTimeMillis();
-
-            if (perfCntr != null)
-                perfCntr.onTaskFinish(info, execEndTs);
-
-            onTaskFinished(new HadoopTaskStatus(state, err, ctx==null ? null : ctx.counters()));
-
-            if (combinerInput != null)
-                combinerInput.close();
-
-            if (ctx != null)
-                ctx.cleanupTaskEnvironment();
-        }
-
-        return null;
-    }
-
-    /**
-     * @param perfCntr Performance counter.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void runTask(HadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
-        if (cancelled)
-            throw new HadoopTaskCancelledException("Task cancelled.");
-
-        try (GridHadoopTaskOutput out = createOutputInternal(ctx);
-             GridHadoopTaskInput in = createInputInternal(ctx)) {
-
-            ctx.input(in);
-            ctx.output(out);
-
-            perfCntr.onTaskStart(ctx.taskInfo(), U.currentTimeMillis());
-
-            ctx.run();
-        }
-    }
-
-    /**
-     * Cancel the executed task.
-     */
-    public void cancel() {
-        cancelled = true;
-
-        if (ctx != null)
-            ctx.cancel();
-    }
-
-    /**
-     * @param status Task status.
-     */
-    protected abstract void onTaskFinished(HadoopTaskStatus status);
-
-    /**
-     * @param ctx Task context.
-     * @return Task input.
-     * @throws IgniteCheckedException If failed.
-     */
-    @SuppressWarnings("unchecked")
-    private GridHadoopTaskInput createInputInternal(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-        switch (ctx.taskInfo().type()) {
-            case SETUP:
-            case MAP:
-            case COMMIT:
-            case ABORT:
-                return null;
-
-            case COMBINE:
-                assert combinerInput != null;
-
-                return combinerInput.input(ctx);
-
-            default:
-                return createInput(ctx);
-        }
-    }
-
-    /**
-     * @param ctx Task context.
-     * @return Input.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected abstract GridHadoopTaskInput createInput(GridHadoopTaskContext ctx) throws IgniteCheckedException;
-
-    /**
-     * @param ctx Task info.
-     * @return Output.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected abstract GridHadoopTaskOutput createOutput(GridHadoopTaskContext ctx) throws IgniteCheckedException;
-
-    /**
-     * @param ctx Task info.
-     * @return Task output.
-     * @throws IgniteCheckedException If failed.
-     */
-    private GridHadoopTaskOutput createOutputInternal(GridHadoopTaskContext ctx) throws IgniteCheckedException {
-        switch (ctx.taskInfo().type()) {
-            case SETUP:
-            case REDUCE:
-            case COMMIT:
-            case ABORT:
-                return null;
-
-            case MAP:
-                if (job.info().hasCombiner()) {
-                    assert combinerInput == null;
-
-                    combinerInput = get(job.info(), SHUFFLE_COMBINER_NO_SORTING, false) ?
-                        new HadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)):
-                        new HadoopSkipList(job.info(), mem); // TODO replace with red-black tree
-
-                    return combinerInput.startAdding(ctx);
-                }
-
-            default:
-                return createOutput(ctx);
-        }
-    }
-
-    /**
-     * @return Task info.
-     */
-    public GridHadoopTaskInfo taskInfo() {
-        return info;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
index 934ff35..e217c57 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
@@ -35,7 +35,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
     private HadoopJobTracker jobTracker;
 
     /** */
-    private final ConcurrentMap<GridHadoopJobId, Collection<GridHadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
+    private final ConcurrentMap<GridHadoopJobId, Collection<HadoopRunnableTask>> jobs = new ConcurrentHashMap<>();
 
     /** Executor service to run tasks. */
     private HadoopExecutorService exec;
@@ -74,22 +74,22 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
             log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() +
                 ", tasksCnt=" + tasks.size() + ']');
 
-        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(job.id());
+        Collection<HadoopRunnableTask> executedTasks = jobs.get(job.id());
 
         if (executedTasks == null) {
             executedTasks = new GridConcurrentHashSet<>();
 
-            Collection<GridHadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks);
+            Collection<HadoopRunnableTask> extractedCol = jobs.put(job.id(), executedTasks);
 
             assert extractedCol == null;
         }
 
-        final Collection<GridHadoopRunnableTask> finalExecutedTasks = executedTasks;
+        final Collection<HadoopRunnableTask> finalExecutedTasks = executedTasks;
 
         for (final GridHadoopTaskInfo info : tasks) {
             assert info != null;
 
-            GridHadoopRunnableTask task = new GridHadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
+            HadoopRunnableTask task = new HadoopRunnableTask(log, job, ctx.shuffle().memory(), info,
                 ctx.localNodeId()) {
                 @Override protected void onTaskFinished(HadoopTaskStatus status) {
                     if (log.isDebugEnabled())
@@ -127,10 +127,10 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param jobId Job ID to cancel.
      */
     @Override public void cancelTasks(GridHadoopJobId jobId) {
-        Collection<GridHadoopRunnableTask> executedTasks = jobs.get(jobId);
+        Collection<HadoopRunnableTask> executedTasks = jobs.get(jobId);
 
         if (executedTasks != null) {
-            for (GridHadoopRunnableTask task : executedTasks)
+            for (HadoopRunnableTask task : executedTasks)
                 task.cancel();
         }
     }
@@ -138,7 +138,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
     /** {@inheritDoc} */
     @Override public void onJobStateChanged(HadoopJobMetadata meta) throws IgniteCheckedException {
         if (meta.phase() == GridHadoopJobPhase.PHASE_COMPLETE) {
-            Collection<GridHadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
+            Collection<HadoopRunnableTask> executedTasks = jobs.remove(meta.jobId());
 
             assert executedTasks == null || executedTasks.isEmpty();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
index 19f903f..d89d7d0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopExecutorService.java
@@ -169,8 +169,8 @@ public class HadoopExecutorService {
     private void startThread(final Callable<?> task) {
         String workerName;
 
-        if (task instanceof GridHadoopRunnableTask) {
-            final GridHadoopTaskInfo i = ((GridHadoopRunnableTask)task).taskInfo();
+        if (task instanceof HadoopRunnableTask) {
+            final GridHadoopTaskInfo i = ((HadoopRunnableTask)task).taskInfo();
 
             workerName = "Hadoop-task-" + i.jobId() + "-" + i.type() + "-" + i.taskNumber() + "-" + i.attempt();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
new file mode 100644
index 0000000..5b10d6f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
@@ -0,0 +1,268 @@
+/*
+ * 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.taskexecutor;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.counter.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.collections.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobProperty.*;
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
+
+/**
+ * Runnable task.
+ */
+public abstract class HadoopRunnableTask implements Callable<Void> {
+    /** */
+    private final GridUnsafeMemory mem;
+
+    /** */
+    private final IgniteLogger log;
+
+    /** */
+    private final GridHadoopJob job;
+
+    /** Task to run. */
+    private final GridHadoopTaskInfo info;
+
+    /** Submit time. */
+    private final long submitTs = U.currentTimeMillis();
+
+    /** Execution start timestamp. */
+    private long execStartTs;
+
+    /** Execution end timestamp. */
+    private long execEndTs;
+
+    /** */
+    private HadoopMultimap combinerInput;
+
+    /** */
+    private volatile GridHadoopTaskContext ctx;
+
+    /** Set if task is to cancelling. */
+    private volatile boolean cancelled;
+
+    /** Node id. */
+    private UUID nodeId;
+
+    /**
+     * @param log Log.
+     * @param job Job.
+     * @param mem Memory.
+     * @param info Task info.
+     * @param nodeId Node id.
+     */
+    protected HadoopRunnableTask(IgniteLogger log, GridHadoopJob job, GridUnsafeMemory mem, GridHadoopTaskInfo info,
+        UUID nodeId) {
+        this.nodeId = nodeId;
+        this.log = log.getLogger(HadoopRunnableTask.class);
+        this.job = job;
+        this.mem = mem;
+        this.info = info;
+    }
+
+    /**
+     * @return Wait time.
+     */
+    public long waitTime() {
+        return execStartTs - submitTs;
+    }
+
+    /**
+     * @return Execution time.
+     */
+    public long executionTime() {
+        return execEndTs - execStartTs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Void call() throws IgniteCheckedException {
+        execStartTs = U.currentTimeMillis();
+
+        Throwable err = null;
+
+        HadoopTaskState state = HadoopTaskState.COMPLETED;
+
+        HadoopPerformanceCounter perfCntr = null;
+
+        try {
+            ctx = job.getTaskContext(info);
+
+            perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
+
+            perfCntr.onTaskSubmit(info, submitTs);
+            perfCntr.onTaskPrepare(info, execStartTs);
+
+            ctx.prepareTaskEnvironment();
+
+            runTask(perfCntr);
+
+            if (info.type() == MAP && job.info().hasCombiner()) {
+                ctx.taskInfo(new GridHadoopTaskInfo(COMBINE, info.jobId(), info.taskNumber(), info.attempt(), null));
+
+                try {
+                    runTask(perfCntr);
+                }
+                finally {
+                    ctx.taskInfo(info);
+                }
+            }
+        }
+        catch (HadoopTaskCancelledException ignored) {
+            state = HadoopTaskState.CANCELED;
+        }
+        catch (Throwable e) {
+            state = HadoopTaskState.FAILED;
+            err = e;
+
+            U.error(log, "Task execution failed.", e);
+        }
+        finally {
+            execEndTs = U.currentTimeMillis();
+
+            if (perfCntr != null)
+                perfCntr.onTaskFinish(info, execEndTs);
+
+            onTaskFinished(new HadoopTaskStatus(state, err, ctx==null ? null : ctx.counters()));
+
+            if (combinerInput != null)
+                combinerInput.close();
+
+            if (ctx != null)
+                ctx.cleanupTaskEnvironment();
+        }
+
+        return null;
+    }
+
+    /**
+     * @param perfCntr Performance counter.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void runTask(HadoopPerformanceCounter perfCntr) throws IgniteCheckedException {
+        if (cancelled)
+            throw new HadoopTaskCancelledException("Task cancelled.");
+
+        try (GridHadoopTaskOutput out = createOutputInternal(ctx);
+             GridHadoopTaskInput in = createInputInternal(ctx)) {
+
+            ctx.input(in);
+            ctx.output(out);
+
+            perfCntr.onTaskStart(ctx.taskInfo(), U.currentTimeMillis());
+
+            ctx.run();
+        }
+    }
+
+    /**
+     * Cancel the executed task.
+     */
+    public void cancel() {
+        cancelled = true;
+
+        if (ctx != null)
+            ctx.cancel();
+    }
+
+    /**
+     * @param status Task status.
+     */
+    protected abstract void onTaskFinished(HadoopTaskStatus status);
+
+    /**
+     * @param ctx Task context.
+     * @return Task input.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private GridHadoopTaskInput createInputInternal(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+        switch (ctx.taskInfo().type()) {
+            case SETUP:
+            case MAP:
+            case COMMIT:
+            case ABORT:
+                return null;
+
+            case COMBINE:
+                assert combinerInput != null;
+
+                return combinerInput.input(ctx);
+
+            default:
+                return createInput(ctx);
+        }
+    }
+
+    /**
+     * @param ctx Task context.
+     * @return Input.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected abstract GridHadoopTaskInput createInput(GridHadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param ctx Task info.
+     * @return Output.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected abstract GridHadoopTaskOutput createOutput(GridHadoopTaskContext ctx) throws IgniteCheckedException;
+
+    /**
+     * @param ctx Task info.
+     * @return Task output.
+     * @throws IgniteCheckedException If failed.
+     */
+    private GridHadoopTaskOutput createOutputInternal(GridHadoopTaskContext ctx) throws IgniteCheckedException {
+        switch (ctx.taskInfo().type()) {
+            case SETUP:
+            case REDUCE:
+            case COMMIT:
+            case ABORT:
+                return null;
+
+            case MAP:
+                if (job.info().hasCombiner()) {
+                    assert combinerInput == null;
+
+                    combinerInput = get(job.info(), SHUFFLE_COMBINER_NO_SORTING, false) ?
+                        new HadoopHashMultimap(job.info(), mem, get(job.info(), COMBINER_HASHMAP_SIZE, 8 * 1024)):
+                        new HadoopSkipList(job.info(), mem); // TODO replace with red-black tree
+
+                    return combinerInput.startAdding(ctx);
+                }
+
+            default:
+                return createOutput(ctx);
+        }
+    }
+
+    /**
+     * @return Task info.
+     */
+    public GridHadoopTaskInfo taskInfo() {
+        return info;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
index 10ad648..f05761e 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
@@ -64,7 +64,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     private String pathSep;
 
     /** Hadoop external communication. */
-    private GridHadoopExternalCommunication comm;
+    private HadoopExternalCommunication comm;
 
     /** Starting processes. */
     private final ConcurrentMap<UUID, HadoopProcess> runningProcsByProcId = new ConcurrentHashMap8<>();
@@ -90,7 +90,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
 
         initJavaCommand();
 
-        comm = new GridHadoopExternalCommunication(
+        comm = new HadoopExternalCommunication(
             ctx.localNodeId(),
             UUID.randomUUID(),
             ctx.kernalContext().config().getMarshaller(),
@@ -508,7 +508,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         cmd.addAll(startMeta.jvmOptions());
         cmd.add("-cp");
         cmd.add(buildClasspath(startMeta.classpath()));
-        cmd.add(GridHadoopExternalProcessStarter.class.getName());
+        cmd.add(HadoopExternalProcessStarter.class.getName());
         cmd.add("-cpid");
         cmd.add(String.valueOf(childProcId));
         cmd.add("-ppid");
@@ -635,7 +635,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     /**
      *
      */
-    private class MessageListener implements GridHadoopMessageListener {
+    private class MessageListener implements HadoopMessageListener {
         /** {@inheritDoc} */
         @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
             if (!busyLock.tryReadLock())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
deleted file mode 100644
index 21552e2..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
+++ /dev/null
@@ -1,440 +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.taskexecutor.external.child;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.shuffle.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.offheap.unsafe.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
-
-/**
- * Hadoop process base.
- */
-@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-public class GridHadoopChildProcessRunner {
-    /** Node process descriptor. */
-    private HadoopProcessDescriptor nodeDesc;
-
-    /** Message processing executor service. */
-    private ExecutorService msgExecSvc;
-
-    /** Task executor service. */
-    private HadoopExecutorService execSvc;
-
-    /** */
-    protected GridUnsafeMemory mem = new GridUnsafeMemory(0);
-
-    /** External communication. */
-    private GridHadoopExternalCommunication comm;
-
-    /** Logger. */
-    private IgniteLogger log;
-
-    /** Init guard. */
-    private final AtomicBoolean initGuard = new AtomicBoolean();
-
-    /** Start time. */
-    private long startTime;
-
-    /** Init future. */
-    private final GridFutureAdapterEx<?> initFut = new GridFutureAdapterEx<>();
-
-    /** Job instance. */
-    private GridHadoopJob job;
-
-    /** Number of uncompleted tasks. */
-    private final AtomicInteger pendingTasks = new AtomicInteger();
-
-    /** Shuffle job. */
-    private HadoopShuffleJob<HadoopProcessDescriptor> shuffleJob;
-
-    /** Concurrent mappers. */
-    private int concMappers;
-
-    /** Concurrent reducers. */
-    private int concReducers;
-
-    /**
-     * Starts child process runner.
-     */
-    public void start(GridHadoopExternalCommunication comm, HadoopProcessDescriptor nodeDesc,
-        ExecutorService msgExecSvc, IgniteLogger parentLog)
-        throws IgniteCheckedException {
-        this.comm = comm;
-        this.nodeDesc = nodeDesc;
-        this.msgExecSvc = msgExecSvc;
-
-        comm.setListener(new MessageListener());
-        log = parentLog.getLogger(GridHadoopChildProcessRunner.class);
-
-        startTime = U.currentTimeMillis();
-
-        // At this point node knows that this process has started.
-        comm.sendMessage(this.nodeDesc, new HadoopProcessStartedAck());
-    }
-
-    /**
-     * Initializes process for task execution.
-     *
-     * @param req Initialization request.
-     */
-    private void prepareProcess(HadoopPrepareForJobRequest req) {
-        if (initGuard.compareAndSet(false, true)) {
-            try {
-                if (log.isDebugEnabled())
-                    log.debug("Initializing external hadoop task: " + req);
-
-                assert job == null;
-
-                job = req.jobInfo().createJob(req.jobId(), log);
-
-                job.initialize(true, nodeDesc.processId());
-
-                shuffleJob = new HadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
-                    req.totalReducerCount(), req.localReducers());
-
-                initializeExecutors(req);
-
-                if (log.isDebugEnabled())
-                    log.debug("External process initialized [initWaitTime=" +
-                        (U.currentTimeMillis() - startTime) + ']');
-
-                initFut.onDone(null, null);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to initialize process: " + req, e);
-
-                initFut.onDone(e);
-            }
-        }
-        else
-            log.warning("Duplicate initialize process request received (will ignore): " + req);
-    }
-
-    /**
-     * @param req Task execution request.
-     */
-    private void runTasks(final HadoopTaskExecutionRequest req) {
-        if (!initFut.isDone() && log.isDebugEnabled())
-            log.debug("Will wait for process initialization future completion: " + req);
-
-        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
-            @Override public void apply(IgniteInternalFuture<?> f) {
-                try {
-                    // Make sure init was successful.
-                    f.get();
-
-                    boolean set = pendingTasks.compareAndSet(0, req.tasks().size());
-
-                    assert set;
-
-                    GridHadoopTaskInfo info = F.first(req.tasks());
-
-                    assert info != null;
-
-                    int size = info.type() == MAP ? concMappers : concReducers;
-
-//                    execSvc.setCorePoolSize(size);
-//                    execSvc.setMaximumPoolSize(size);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Set executor service size for task type [type=" + info.type() +
-                            ", size=" + size + ']');
-
-                    for (GridHadoopTaskInfo taskInfo : req.tasks()) {
-                        if (log.isDebugEnabled())
-                            log.debug("Submitted task for external execution: " + taskInfo);
-
-                        execSvc.submit(new GridHadoopRunnableTask(log, job, mem, taskInfo, nodeDesc.parentNodeId()) {
-                            @Override protected void onTaskFinished(HadoopTaskStatus status) {
-                                onTaskFinished0(this, status);
-                            }
-
-                            @Override protected GridHadoopTaskInput createInput(GridHadoopTaskContext ctx)
-                                throws IgniteCheckedException {
-                                return shuffleJob.input(ctx);
-                            }
-
-                            @Override protected GridHadoopTaskOutput createOutput(GridHadoopTaskContext ctx)
-                                throws IgniteCheckedException {
-                                return shuffleJob.output(ctx);
-                            }
-                        });
-                    }
-                }
-                catch (IgniteCheckedException e) {
-                    for (GridHadoopTaskInfo info : req.tasks())
-                        notifyTaskFinished(info, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
-                }
-            }
-        });
-    }
-
-    /**
-     * Creates executor services.
-     *
-     * @param req Init child process request.
-     */
-    private void initializeExecutors(HadoopPrepareForJobRequest req) {
-        int cpus = Runtime.getRuntime().availableProcessors();
-//
-//        concMappers = get(req.jobInfo(), EXTERNAL_CONCURRENT_MAPPERS, cpus);
-//        concReducers = get(req.jobInfo(), EXTERNAL_CONCURRENT_REDUCERS, cpus);
-
-        execSvc = new HadoopExecutorService(log, "", cpus * 2, 1024);
-    }
-
-    /**
-     * Updates external process map so that shuffle can proceed with sending messages to reducers.
-     *
-     * @param req Update request.
-     */
-    private void updateTasks(final HadoopJobInfoUpdateRequest req) {
-        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
-            @Override public void apply(IgniteInternalFuture<?> gridFut) {
-                assert initGuard.get();
-
-                assert req.jobId().equals(job.id());
-
-                if (req.reducersAddresses() != null) {
-                    if (shuffleJob.initializeReduceAddresses(req.reducersAddresses())) {
-                        shuffleJob.startSending("external",
-                            new IgniteInClosure2X<HadoopProcessDescriptor, HadoopShuffleMessage>() {
-                                @Override public void applyx(HadoopProcessDescriptor dest,
-                                    HadoopShuffleMessage msg) throws IgniteCheckedException {
-                                    comm.sendMessage(dest, msg);
-                                }
-                            });
-                    }
-                }
-            }
-        });
-    }
-
-    /**
-     * Stops all executors and running tasks.
-     */
-    private void shutdown() {
-        if (execSvc != null)
-            execSvc.shutdown(5000);
-
-        if (msgExecSvc != null)
-            msgExecSvc.shutdownNow();
-
-        try {
-            job.dispose(true);
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to dispose job.", e);
-        }
-    }
-
-    /**
-     * Notifies node about task finish.
-     *
-     * @param run Finished task runnable.
-     * @param status Task status.
-     */
-    private void onTaskFinished0(GridHadoopRunnableTask run, HadoopTaskStatus status) {
-        GridHadoopTaskInfo info = run.taskInfo();
-
-        int pendingTasks0 = pendingTasks.decrementAndGet();
-
-        if (log.isDebugEnabled())
-            log.debug("Hadoop task execution finished [info=" + info
-                + ", state=" + status.state() + ", waitTime=" + run.waitTime() + ", execTime=" + run.executionTime() +
-                ", pendingTasks=" + pendingTasks0 +
-                ", err=" + status.failCause() + ']');
-
-        assert info.type() == MAP || info.type() == REDUCE : "Only MAP or REDUCE tasks are supported.";
-
-        boolean flush = pendingTasks0 == 0 && info.type() == MAP;
-
-        notifyTaskFinished(info, status, flush);
-    }
-
-    /**
-     * @param taskInfo Finished task info.
-     * @param status Task status.
-     */
-    private void notifyTaskFinished(final GridHadoopTaskInfo taskInfo, final HadoopTaskStatus status,
-        boolean flush) {
-
-        final HadoopTaskState state = status.state();
-        final Throwable err = status.failCause();
-
-        if (!flush) {
-            try {
-                if (log.isDebugEnabled())
-                    log.debug("Sending notification to parent node [taskInfo=" + taskInfo + ", state=" + state +
-                        ", err=" + err + ']');
-
-                comm.sendMessage(nodeDesc, new HadoopTaskFinishedMessage(taskInfo, status));
-            }
-            catch (IgniteCheckedException e) {
-                log.error("Failed to send message to parent node (will terminate child process).", e);
-
-                shutdown();
-
-                terminate();
-            }
-        }
-        else {
-            if (log.isDebugEnabled())
-                log.debug("Flushing shuffle messages before sending last task completion notification [taskInfo=" +
-                    taskInfo + ", state=" + state + ", err=" + err + ']');
-
-            final long start = U.currentTimeMillis();
-
-            try {
-                shuffleJob.flush().listenAsync(new CI1<IgniteInternalFuture<?>>() {
-                    @Override public void apply(IgniteInternalFuture<?> f) {
-                        long end = U.currentTimeMillis();
-
-                        if (log.isDebugEnabled())
-                            log.debug("Finished flushing shuffle messages [taskInfo=" + taskInfo +
-                                ", flushTime=" + (end - start) + ']');
-
-                        try {
-                            // Check for errors on shuffle.
-                            f.get();
-
-                            notifyTaskFinished(taskInfo, status, false);
-                        }
-                        catch (IgniteCheckedException e) {
-                            log.error("Failed to flush shuffle messages (will fail the task) [taskInfo=" + taskInfo +
-                                ", state=" + state + ", err=" + err + ']', e);
-
-                            notifyTaskFinished(taskInfo,
-                                new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
-                        }
-                    }
-                });
-            }
-            catch (IgniteCheckedException e) {
-                log.error("Failed to flush shuffle messages (will fail the task) [taskInfo=" + taskInfo +
-                    ", state=" + state + ", err=" + err + ']', e);
-
-                notifyTaskFinished(taskInfo, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
-            }
-        }
-    }
-
-    /**
-     * Checks if message was received from parent node and prints warning if not.
-     *
-     * @param desc Sender process ID.
-     * @param msg Received message.
-     * @return {@code True} if received from parent node.
-     */
-    private boolean validateNodeMessage(HadoopProcessDescriptor desc, HadoopMessage msg) {
-        if (!nodeDesc.processId().equals(desc.processId())) {
-            log.warning("Received process control request from unknown process (will ignore) [desc=" + desc +
-                ", msg=" + msg + ']');
-
-            return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * Stops execution of this process.
-     */
-    private void terminate() {
-        System.exit(1);
-    }
-
-    /**
-     * Message listener.
-     */
-    private class MessageListener implements GridHadoopMessageListener {
-        /** {@inheritDoc} */
-        @Override public void onMessageReceived(final HadoopProcessDescriptor desc, final HadoopMessage msg) {
-            if (msg instanceof HadoopTaskExecutionRequest) {
-                if (validateNodeMessage(desc, msg))
-                    runTasks((HadoopTaskExecutionRequest)msg);
-            }
-            else if (msg instanceof HadoopJobInfoUpdateRequest) {
-                if (validateNodeMessage(desc, msg))
-                    updateTasks((HadoopJobInfoUpdateRequest)msg);
-            }
-            else if (msg instanceof HadoopPrepareForJobRequest) {
-                if (validateNodeMessage(desc, msg))
-                    prepareProcess((HadoopPrepareForJobRequest)msg);
-            }
-            else if (msg instanceof HadoopShuffleMessage) {
-                if (log.isTraceEnabled())
-                    log.trace("Received shuffle message [desc=" + desc + ", msg=" + msg + ']');
-
-                initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
-                    @Override public void apply(IgniteInternalFuture<?> f) {
-                        try {
-                            HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
-
-                            shuffleJob.onShuffleMessage(m);
-
-                            comm.sendMessage(desc, new HadoopShuffleAck(m.id(), m.jobId()));
-                        }
-                        catch (IgniteCheckedException e) {
-                            U.error(log, "Failed to process hadoop shuffle message [desc=" + desc + ", msg=" + msg + ']', e);
-                        }
-                    }
-                });
-            }
-            else if (msg instanceof HadoopShuffleAck) {
-                if (log.isTraceEnabled())
-                    log.trace("Received shuffle ack [desc=" + desc + ", msg=" + msg + ']');
-
-                shuffleJob.onShuffleAck((HadoopShuffleAck)msg);
-            }
-            else
-                log.warning("Unknown message received (will ignore) [desc=" + desc + ", msg=" + msg + ']');
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
-            if (log.isDebugEnabled())
-                log.debug("Lost connection with remote process: " + desc);
-
-            if (desc == null)
-                U.warn(log, "Handshake failed.");
-            else if (desc.processId().equals(nodeDesc.processId())) {
-                log.warning("Child process lost connection with parent node (will terminate child process).");
-
-                shutdown();
-
-                terminate();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
deleted file mode 100644
index 1216c9a..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
+++ /dev/null
@@ -1,296 +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.taskexecutor.external.child;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.logger.log4j.*;
-import org.apache.ignite.marshaller.optimized.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Hadoop external process base class.
- */
-public class GridHadoopExternalProcessStarter {
-    /** Path to Log4j configuration file. */
-    public static final String DFLT_LOG4J_CONFIG = "config/ignite-log4j.xml";
-
-    /** Arguments. */
-    private Args args;
-
-    /** System out. */
-    private OutputStream out;
-
-    /** System err. */
-    private OutputStream err;
-
-    /**
-     * @param args Parsed arguments.
-     */
-    public GridHadoopExternalProcessStarter(Args args) {
-        this.args = args;
-    }
-
-    /**
-     * @param cmdArgs Process arguments.
-     */
-    public static void main(String[] cmdArgs) {
-        try {
-            Args args = arguments(cmdArgs);
-
-            new GridHadoopExternalProcessStarter(args).run();
-        }
-        catch (Exception e) {
-            System.err.println("Failed");
-
-            System.err.println(e.getMessage());
-
-            e.printStackTrace(System.err);
-        }
-    }
-
-    /**
-     *
-     * @throws Exception
-     */
-    public void run() throws Exception {
-        U.setWorkDirectory(args.workDir, U.getIgniteHome());
-
-        File outputDir = outputDirectory();
-
-        initializeStreams(outputDir);
-
-        ExecutorService msgExecSvc = Executors.newFixedThreadPool(
-            Integer.getInteger("MSG_THREAD_POOL_SIZE", Runtime.getRuntime().availableProcessors() * 2));
-
-        IgniteLogger log = logger(outputDir);
-
-        GridHadoopExternalCommunication comm = new GridHadoopExternalCommunication(
-            args.nodeId,
-            args.childProcId,
-            new OptimizedMarshaller(),
-            log,
-            msgExecSvc,
-            "external"
-        );
-
-        comm.start();
-
-        HadoopProcessDescriptor nodeDesc = new HadoopProcessDescriptor(args.nodeId, args.parentProcId);
-        nodeDesc.address(args.addr);
-        nodeDesc.tcpPort(args.tcpPort);
-        nodeDesc.sharedMemoryPort(args.shmemPort);
-
-        GridHadoopChildProcessRunner runner = new GridHadoopChildProcessRunner();
-
-        runner.start(comm, nodeDesc, msgExecSvc, log);
-
-        System.err.println("Started");
-        System.err.flush();
-
-        System.setOut(new PrintStream(out));
-        System.setErr(new PrintStream(err));
-    }
-
-    /**
-     * @param outputDir Directory for process output.
-     * @throws Exception
-     */
-    private void initializeStreams(File outputDir) throws Exception {
-        out = new FileOutputStream(new File(outputDir, args.childProcId + ".out"));
-        err = new FileOutputStream(new File(outputDir, args.childProcId + ".err"));
-    }
-
-    /**
-     * @return Path to output directory.
-     * @throws IOException If failed.
-     */
-    private File outputDirectory() throws IOException {
-        File f = new File(args.out);
-
-        if (!f.exists()) {
-            if (!f.mkdirs())
-                throw new IOException("Failed to create output directory: " + args.out);
-        }
-        else {
-            if (f.isFile())
-                throw new IOException("Output directory is a file: " + args.out);
-        }
-
-        return f;
-    }
-
-    /**
-     * @param outputDir Directory for process output.
-     * @return Logger.
-     */
-    private IgniteLogger logger(final File outputDir) {
-        final URL url = U.resolveIgniteUrl(DFLT_LOG4J_CONFIG);
-
-        Log4JLogger logger;
-
-        try {
-            logger = url != null ? new Log4JLogger(url) : new Log4JLogger(true);
-        }
-        catch (IgniteCheckedException e) {
-            System.err.println("Failed to create URL-based logger. Will use default one.");
-
-            e.printStackTrace();
-
-            logger = new Log4JLogger(true);
-        }
-
-        logger.updateFilePath(new IgniteClosure<String, String>() {
-            @Override public String apply(String s) {
-                return new File(outputDir, args.childProcId + ".log").getAbsolutePath();
-            }
-        });
-
-        return logger;
-    }
-
-    /**
-     * @param processArgs Process arguments.
-     * @return Child process instance.
-     */
-    private static Args arguments(String[] processArgs) throws Exception {
-        Args args = new Args();
-
-        for (int i = 0; i < processArgs.length; i++) {
-            String arg = processArgs[i];
-
-            switch (arg) {
-                case "-cpid": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing process ID for '-cpid' parameter");
-
-                    String procIdStr = processArgs[++i];
-
-                    args.childProcId = UUID.fromString(procIdStr);
-
-                    break;
-                }
-
-                case "-ppid": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing process ID for '-ppid' parameter");
-
-                    String procIdStr = processArgs[++i];
-
-                    args.parentProcId = UUID.fromString(procIdStr);
-
-                    break;
-                }
-
-                case "-nid": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing node ID for '-nid' parameter");
-
-                    String nodeIdStr = processArgs[++i];
-
-                    args.nodeId = UUID.fromString(nodeIdStr);
-
-                    break;
-                }
-
-                case "-addr": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing node address for '-addr' parameter");
-
-                    args.addr = processArgs[++i];
-
-                    break;
-                }
-
-                case "-tport": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing tcp port for '-tport' parameter");
-
-                    args.tcpPort = Integer.parseInt(processArgs[++i]);
-
-                    break;
-                }
-
-                case "-sport": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing shared memory port for '-sport' parameter");
-
-                    args.shmemPort = Integer.parseInt(processArgs[++i]);
-
-                    break;
-                }
-
-                case "-out": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing output folder name for '-out' parameter");
-
-                    args.out = processArgs[++i];
-
-                    break;
-                }
-
-                case "-wd": {
-                    if (i == processArgs.length - 1)
-                        throw new Exception("Missing work folder name for '-wd' parameter");
-
-                    args.workDir = processArgs[++i];
-
-                    break;
-                }
-            }
-        }
-
-        return args;
-    }
-
-    /**
-     * Execution arguments.
-     */
-    private static class Args {
-        /** Process ID. */
-        private UUID childProcId;
-
-        /** Process ID. */
-        private UUID parentProcId;
-
-        /** Process ID. */
-        private UUID nodeId;
-
-        /** Node address. */
-        private String addr;
-
-        /** TCP port */
-        private int tcpPort;
-
-        /** Shmem port. */
-        private int shmemPort = -1;
-
-        /** Output folder. */
-        private String out;
-
-        /** Work directory. */
-        private String workDir;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1c4b00d4/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
new file mode 100644
index 0000000..6345704
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -0,0 +1,440 @@
+/*
+ * 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.taskexecutor.external.child;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.shuffle.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.offheap.unsafe.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
+
+/**
+ * Hadoop process base.
+ */
+@SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+public class HadoopChildProcessRunner {
+    /** Node process descriptor. */
+    private HadoopProcessDescriptor nodeDesc;
+
+    /** Message processing executor service. */
+    private ExecutorService msgExecSvc;
+
+    /** Task executor service. */
+    private HadoopExecutorService execSvc;
+
+    /** */
+    protected GridUnsafeMemory mem = new GridUnsafeMemory(0);
+
+    /** External communication. */
+    private HadoopExternalCommunication comm;
+
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** Init guard. */
+    private final AtomicBoolean initGuard = new AtomicBoolean();
+
+    /** Start time. */
+    private long startTime;
+
+    /** Init future. */
+    private final GridFutureAdapterEx<?> initFut = new GridFutureAdapterEx<>();
+
+    /** Job instance. */
+    private GridHadoopJob job;
+
+    /** Number of uncompleted tasks. */
+    private final AtomicInteger pendingTasks = new AtomicInteger();
+
+    /** Shuffle job. */
+    private HadoopShuffleJob<HadoopProcessDescriptor> shuffleJob;
+
+    /** Concurrent mappers. */
+    private int concMappers;
+
+    /** Concurrent reducers. */
+    private int concReducers;
+
+    /**
+     * Starts child process runner.
+     */
+    public void start(HadoopExternalCommunication comm, HadoopProcessDescriptor nodeDesc,
+        ExecutorService msgExecSvc, IgniteLogger parentLog)
+        throws IgniteCheckedException {
+        this.comm = comm;
+        this.nodeDesc = nodeDesc;
+        this.msgExecSvc = msgExecSvc;
+
+        comm.setListener(new MessageListener());
+        log = parentLog.getLogger(HadoopChildProcessRunner.class);
+
+        startTime = U.currentTimeMillis();
+
+        // At this point node knows that this process has started.
+        comm.sendMessage(this.nodeDesc, new HadoopProcessStartedAck());
+    }
+
+    /**
+     * Initializes process for task execution.
+     *
+     * @param req Initialization request.
+     */
+    private void prepareProcess(HadoopPrepareForJobRequest req) {
+        if (initGuard.compareAndSet(false, true)) {
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Initializing external hadoop task: " + req);
+
+                assert job == null;
+
+                job = req.jobInfo().createJob(req.jobId(), log);
+
+                job.initialize(true, nodeDesc.processId());
+
+                shuffleJob = new HadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
+                    req.totalReducerCount(), req.localReducers());
+
+                initializeExecutors(req);
+
+                if (log.isDebugEnabled())
+                    log.debug("External process initialized [initWaitTime=" +
+                        (U.currentTimeMillis() - startTime) + ']');
+
+                initFut.onDone(null, null);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to initialize process: " + req, e);
+
+                initFut.onDone(e);
+            }
+        }
+        else
+            log.warning("Duplicate initialize process request received (will ignore): " + req);
+    }
+
+    /**
+     * @param req Task execution request.
+     */
+    private void runTasks(final HadoopTaskExecutionRequest req) {
+        if (!initFut.isDone() && log.isDebugEnabled())
+            log.debug("Will wait for process initialization future completion: " + req);
+
+        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> f) {
+                try {
+                    // Make sure init was successful.
+                    f.get();
+
+                    boolean set = pendingTasks.compareAndSet(0, req.tasks().size());
+
+                    assert set;
+
+                    GridHadoopTaskInfo info = F.first(req.tasks());
+
+                    assert info != null;
+
+                    int size = info.type() == MAP ? concMappers : concReducers;
+
+//                    execSvc.setCorePoolSize(size);
+//                    execSvc.setMaximumPoolSize(size);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Set executor service size for task type [type=" + info.type() +
+                            ", size=" + size + ']');
+
+                    for (GridHadoopTaskInfo taskInfo : req.tasks()) {
+                        if (log.isDebugEnabled())
+                            log.debug("Submitted task for external execution: " + taskInfo);
+
+                        execSvc.submit(new HadoopRunnableTask(log, job, mem, taskInfo, nodeDesc.parentNodeId()) {
+                            @Override protected void onTaskFinished(HadoopTaskStatus status) {
+                                onTaskFinished0(this, status);
+                            }
+
+                            @Override protected GridHadoopTaskInput createInput(GridHadoopTaskContext ctx)
+                                throws IgniteCheckedException {
+                                return shuffleJob.input(ctx);
+                            }
+
+                            @Override protected GridHadoopTaskOutput createOutput(GridHadoopTaskContext ctx)
+                                throws IgniteCheckedException {
+                                return shuffleJob.output(ctx);
+                            }
+                        });
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    for (GridHadoopTaskInfo info : req.tasks())
+                        notifyTaskFinished(info, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
+                }
+            }
+        });
+    }
+
+    /**
+     * Creates executor services.
+     *
+     * @param req Init child process request.
+     */
+    private void initializeExecutors(HadoopPrepareForJobRequest req) {
+        int cpus = Runtime.getRuntime().availableProcessors();
+//
+//        concMappers = get(req.jobInfo(), EXTERNAL_CONCURRENT_MAPPERS, cpus);
+//        concReducers = get(req.jobInfo(), EXTERNAL_CONCURRENT_REDUCERS, cpus);
+
+        execSvc = new HadoopExecutorService(log, "", cpus * 2, 1024);
+    }
+
+    /**
+     * Updates external process map so that shuffle can proceed with sending messages to reducers.
+     *
+     * @param req Update request.
+     */
+    private void updateTasks(final HadoopJobInfoUpdateRequest req) {
+        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> gridFut) {
+                assert initGuard.get();
+
+                assert req.jobId().equals(job.id());
+
+                if (req.reducersAddresses() != null) {
+                    if (shuffleJob.initializeReduceAddresses(req.reducersAddresses())) {
+                        shuffleJob.startSending("external",
+                            new IgniteInClosure2X<HadoopProcessDescriptor, HadoopShuffleMessage>() {
+                                @Override public void applyx(HadoopProcessDescriptor dest,
+                                    HadoopShuffleMessage msg) throws IgniteCheckedException {
+                                    comm.sendMessage(dest, msg);
+                                }
+                            });
+                    }
+                }
+            }
+        });
+    }
+
+    /**
+     * Stops all executors and running tasks.
+     */
+    private void shutdown() {
+        if (execSvc != null)
+            execSvc.shutdown(5000);
+
+        if (msgExecSvc != null)
+            msgExecSvc.shutdownNow();
+
+        try {
+            job.dispose(true);
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to dispose job.", e);
+        }
+    }
+
+    /**
+     * Notifies node about task finish.
+     *
+     * @param run Finished task runnable.
+     * @param status Task status.
+     */
+    private void onTaskFinished0(HadoopRunnableTask run, HadoopTaskStatus status) {
+        GridHadoopTaskInfo info = run.taskInfo();
+
+        int pendingTasks0 = pendingTasks.decrementAndGet();
+
+        if (log.isDebugEnabled())
+            log.debug("Hadoop task execution finished [info=" + info
+                + ", state=" + status.state() + ", waitTime=" + run.waitTime() + ", execTime=" + run.executionTime() +
+                ", pendingTasks=" + pendingTasks0 +
+                ", err=" + status.failCause() + ']');
+
+        assert info.type() == MAP || info.type() == REDUCE : "Only MAP or REDUCE tasks are supported.";
+
+        boolean flush = pendingTasks0 == 0 && info.type() == MAP;
+
+        notifyTaskFinished(info, status, flush);
+    }
+
+    /**
+     * @param taskInfo Finished task info.
+     * @param status Task status.
+     */
+    private void notifyTaskFinished(final GridHadoopTaskInfo taskInfo, final HadoopTaskStatus status,
+        boolean flush) {
+
+        final HadoopTaskState state = status.state();
+        final Throwable err = status.failCause();
+
+        if (!flush) {
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Sending notification to parent node [taskInfo=" + taskInfo + ", state=" + state +
+                        ", err=" + err + ']');
+
+                comm.sendMessage(nodeDesc, new HadoopTaskFinishedMessage(taskInfo, status));
+            }
+            catch (IgniteCheckedException e) {
+                log.error("Failed to send message to parent node (will terminate child process).", e);
+
+                shutdown();
+
+                terminate();
+            }
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Flushing shuffle messages before sending last task completion notification [taskInfo=" +
+                    taskInfo + ", state=" + state + ", err=" + err + ']');
+
+            final long start = U.currentTimeMillis();
+
+            try {
+                shuffleJob.flush().listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
+                        long end = U.currentTimeMillis();
+
+                        if (log.isDebugEnabled())
+                            log.debug("Finished flushing shuffle messages [taskInfo=" + taskInfo +
+                                ", flushTime=" + (end - start) + ']');
+
+                        try {
+                            // Check for errors on shuffle.
+                            f.get();
+
+                            notifyTaskFinished(taskInfo, status, false);
+                        }
+                        catch (IgniteCheckedException e) {
+                            log.error("Failed to flush shuffle messages (will fail the task) [taskInfo=" + taskInfo +
+                                ", state=" + state + ", err=" + err + ']', e);
+
+                            notifyTaskFinished(taskInfo,
+                                new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
+                        }
+                    }
+                });
+            }
+            catch (IgniteCheckedException e) {
+                log.error("Failed to flush shuffle messages (will fail the task) [taskInfo=" + taskInfo +
+                    ", state=" + state + ", err=" + err + ']', e);
+
+                notifyTaskFinished(taskInfo, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
+            }
+        }
+    }
+
+    /**
+     * Checks if message was received from parent node and prints warning if not.
+     *
+     * @param desc Sender process ID.
+     * @param msg Received message.
+     * @return {@code True} if received from parent node.
+     */
+    private boolean validateNodeMessage(HadoopProcessDescriptor desc, HadoopMessage msg) {
+        if (!nodeDesc.processId().equals(desc.processId())) {
+            log.warning("Received process control request from unknown process (will ignore) [desc=" + desc +
+                ", msg=" + msg + ']');
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Stops execution of this process.
+     */
+    private void terminate() {
+        System.exit(1);
+    }
+
+    /**
+     * Message listener.
+     */
+    private class MessageListener implements HadoopMessageListener {
+        /** {@inheritDoc} */
+        @Override public void onMessageReceived(final HadoopProcessDescriptor desc, final HadoopMessage msg) {
+            if (msg instanceof HadoopTaskExecutionRequest) {
+                if (validateNodeMessage(desc, msg))
+                    runTasks((HadoopTaskExecutionRequest)msg);
+            }
+            else if (msg instanceof HadoopJobInfoUpdateRequest) {
+                if (validateNodeMessage(desc, msg))
+                    updateTasks((HadoopJobInfoUpdateRequest)msg);
+            }
+            else if (msg instanceof HadoopPrepareForJobRequest) {
+                if (validateNodeMessage(desc, msg))
+                    prepareProcess((HadoopPrepareForJobRequest)msg);
+            }
+            else if (msg instanceof HadoopShuffleMessage) {
+                if (log.isTraceEnabled())
+                    log.trace("Received shuffle message [desc=" + desc + ", msg=" + msg + ']');
+
+                initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
+                        try {
+                            HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
+
+                            shuffleJob.onShuffleMessage(m);
+
+                            comm.sendMessage(desc, new HadoopShuffleAck(m.id(), m.jobId()));
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to process hadoop shuffle message [desc=" + desc + ", msg=" + msg + ']', e);
+                        }
+                    }
+                });
+            }
+            else if (msg instanceof HadoopShuffleAck) {
+                if (log.isTraceEnabled())
+                    log.trace("Received shuffle ack [desc=" + desc + ", msg=" + msg + ']');
+
+                shuffleJob.onShuffleAck((HadoopShuffleAck)msg);
+            }
+            else
+                log.warning("Unknown message received (will ignore) [desc=" + desc + ", msg=" + msg + ']');
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
+            if (log.isDebugEnabled())
+                log.debug("Lost connection with remote process: " + desc);
+
+            if (desc == null)
+                U.warn(log, "Handshake failed.");
+            else if (desc.processId().equals(nodeDesc.processId())) {
+                log.warning("Child process lost connection with parent node (will terminate child process).");
+
+                shutdown();
+
+                terminate();
+            }
+        }
+    }
+}


[13/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
deleted file mode 100644
index 342cbab..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
+++ /dev/null
@@ -1,434 +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.planner;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.igfs.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.igfs.hadoop.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.resources.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-import static org.apache.ignite.IgniteFs.*;
-
-/**
- * Default map-reduce planner implementation.
- */
-public class GridHadoopDefaultMapReducePlanner implements GridHadoopMapReducePlanner {
-    /** Injected grid. */
-    @IgniteInstanceResource
-    private Ignite ignite;
-
-    /** Logger. */
-    @SuppressWarnings("UnusedDeclaration")
-    @LoggerResource
-    private IgniteLogger log;
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopMapReducePlan preparePlan(GridHadoopJob job, Collection<ClusterNode> top,
-        @Nullable GridHadoopMapReducePlan oldPlan) throws IgniteCheckedException {
-        // Convert collection of topology nodes to collection of topology node IDs.
-        Collection<UUID> topIds = new HashSet<>(top.size(), 1.0f);
-
-        for (ClusterNode topNode : top)
-            topIds.add(topNode.id());
-
-        Map<UUID, Collection<GridHadoopInputSplit>> mappers = mappers(top, topIds, job.input());
-
-        int rdcCnt = job.info().reducers();
-
-        if (rdcCnt < 0)
-            throw new IgniteCheckedException("Number of reducers must be non-negative, actual: " + rdcCnt);
-
-        Map<UUID, int[]> reducers = reducers(top, mappers, rdcCnt);
-
-        return new GridHadoopDefaultMapReducePlan(mappers, reducers);
-    }
-
-    /**
-     * Create plan for mappers.
-     *
-     * @param top Topology nodes.
-     * @param topIds Topology node IDs.
-     * @param splits Splits.
-     * @return Mappers map.
-     * @throws IgniteCheckedException If failed.
-     */
-    private Map<UUID, Collection<GridHadoopInputSplit>> mappers(Collection<ClusterNode> top, Collection<UUID> topIds,
-        Iterable<GridHadoopInputSplit> splits) throws IgniteCheckedException {
-        Map<UUID, Collection<GridHadoopInputSplit>> mappers = new HashMap<>();
-
-        Map<String, Collection<UUID>> nodes = hosts(top);
-
-        Map<UUID, Integer> nodeLoads = new HashMap<>(top.size(), 1.0f); // Track node load.
-
-        for (UUID nodeId : topIds)
-            nodeLoads.put(nodeId, 0);
-
-        for (GridHadoopInputSplit split : splits) {
-            UUID nodeId = nodeForSplit(split, topIds, nodes, nodeLoads);
-
-            if (log.isDebugEnabled())
-                log.debug("Mapped split to node [split=" + split + ", nodeId=" + nodeId + ']');
-
-            Collection<GridHadoopInputSplit> nodeSplits = mappers.get(nodeId);
-
-            if (nodeSplits == null) {
-                nodeSplits = new ArrayList<>();
-
-                mappers.put(nodeId, nodeSplits);
-            }
-
-            nodeSplits.add(split);
-
-            // Updated node load.
-            nodeLoads.put(nodeId, nodeLoads.get(nodeId) + 1);
-        }
-
-        return mappers;
-    }
-
-    /**
-     * Groups nodes by host names.
-     *
-     * @param top Topology to group.
-     * @return Map.
-     */
-    private static Map<String, Collection<UUID>> hosts(Collection<ClusterNode> top) {
-        Map<String, Collection<UUID>> grouped = U.newHashMap(top.size());
-
-        for (ClusterNode node : top) {
-            for (String host : node.hostNames()) {
-                Collection<UUID> nodeIds = grouped.get(host);
-
-                if (nodeIds == null) {
-                    // Expecting 1-2 nodes per host.
-                    nodeIds = new ArrayList<>(2);
-
-                    grouped.put(host, nodeIds);
-                }
-
-                nodeIds.add(node.id());
-            }
-        }
-
-        return grouped;
-    }
-
-    /**
-     * Determine the best node for this split.
-     *
-     * @param split Split.
-     * @param topIds Topology node IDs.
-     * @param nodes Nodes.
-     * @param nodeLoads Node load tracker.
-     * @return Node ID.
-     */
-    @SuppressWarnings("unchecked")
-    private UUID nodeForSplit(GridHadoopInputSplit split, Collection<UUID> topIds, Map<String, Collection<UUID>> nodes,
-        Map<UUID, Integer> nodeLoads) throws IgniteCheckedException {
-        if (split instanceof GridHadoopFileBlock) {
-            GridHadoopFileBlock split0 = (GridHadoopFileBlock)split;
-
-            if (IGFS_SCHEME.equalsIgnoreCase(split0.file().getScheme())) {
-                IgfsHadoopEndpoint endpoint = new IgfsHadoopEndpoint(split0.file().getAuthority());
-
-                IgfsEx igfs = null;
-
-                if (F.eq(ignite.name(), endpoint.grid()))
-                    igfs = (IgfsEx)((IgniteEx)ignite).igfsx(endpoint.igfs());
-
-                if (igfs != null && !igfs.isProxy(split0.file())) {
-                    Collection<IgfsBlockLocation> blocks;
-
-                    try {
-                        blocks = igfs.affinity(new IgfsPath(split0.file()), split0.start(), split0.length());
-                    }
-                    catch (IgniteException e) {
-                        throw new IgniteCheckedException(e);
-                    }
-
-                    assert blocks != null;
-
-                    if (blocks.size() == 1)
-                        // Fast-path, split consists of one IGFS block (as in most cases).
-                        return bestNode(blocks.iterator().next().nodeIds(), topIds, nodeLoads, false);
-                    else {
-                        // Slow-path, file consists of multiple IGFS blocks. First, find the most co-located nodes.
-                        Map<UUID, Long> nodeMap = new HashMap<>();
-
-                        List<UUID> bestNodeIds = null;
-                        long bestLen = -1L;
-
-                        for (IgfsBlockLocation block : blocks) {
-                            for (UUID blockNodeId : block.nodeIds()) {
-                                if (topIds.contains(blockNodeId)) {
-                                    Long oldLen = nodeMap.get(blockNodeId);
-                                    long newLen = oldLen == null ? block.length() : oldLen + block.length();
-
-                                    nodeMap.put(blockNodeId, newLen);
-
-                                    if (bestNodeIds == null || bestLen < newLen) {
-                                        bestNodeIds = new ArrayList<>(1);
-
-                                        bestNodeIds.add(blockNodeId);
-
-                                        bestLen = newLen;
-                                    }
-                                    else if (bestLen == newLen) {
-                                        assert !F.isEmpty(bestNodeIds);
-
-                                        bestNodeIds.add(blockNodeId);
-                                    }
-                                }
-                            }
-                        }
-
-                        if (bestNodeIds != null) {
-                            return bestNodeIds.size() == 1 ? bestNodeIds.get(0) :
-                                bestNode(bestNodeIds, topIds, nodeLoads, true);
-                        }
-                    }
-                }
-            }
-        }
-
-        // Cannot use local IGFS for some reason, try selecting the node by host.
-        Collection<UUID> blockNodes = null;
-
-        for (String host : split.hosts()) {
-            Collection<UUID> hostNodes = nodes.get(host);
-
-            if (!F.isEmpty(hostNodes)) {
-                if (blockNodes == null)
-                    blockNodes = new ArrayList<>(hostNodes);
-                else
-                    blockNodes.addAll(hostNodes);
-            }
-        }
-
-        return bestNode(blockNodes, topIds, nodeLoads, false);
-    }
-
-    /**
-     * Finds the best (the least loaded) node among the candidates.
-     *
-     * @param candidates Candidates.
-     * @param topIds Topology node IDs.
-     * @param nodeLoads Known node loads.
-     * @param skipTopCheck Whether to skip topology check.
-     * @return The best node.
-     */
-    private UUID bestNode(@Nullable Collection<UUID> candidates, Collection<UUID> topIds, Map<UUID, Integer> nodeLoads,
-        boolean skipTopCheck) {
-        UUID bestNode = null;
-        int bestLoad = Integer.MAX_VALUE;
-
-        if (candidates != null) {
-            for (UUID candidate : candidates) {
-                if (skipTopCheck || topIds.contains(candidate)) {
-                    int load = nodeLoads.get(candidate);
-
-                    if (bestNode == null || bestLoad > load) {
-                        bestNode = candidate;
-                        bestLoad = load;
-
-                        if (bestLoad == 0)
-                            break; // Minimum load possible, no need for further iterations.
-                    }
-                }
-            }
-        }
-
-        if (bestNode == null) {
-            // Blocks are located on nodes which are not Hadoop-enabled, assign to the least loaded one.
-            bestLoad = Integer.MAX_VALUE;
-
-            for (UUID nodeId : topIds) {
-                int load = nodeLoads.get(nodeId);
-
-                if (bestNode == null || bestLoad > load) {
-                    bestNode = nodeId;
-                    bestLoad = load;
-
-                    if (bestLoad == 0)
-                        break; // Minimum load possible, no need for further iterations.
-                }
-            }
-        }
-
-        assert bestNode != null;
-
-        return bestNode;
-    }
-
-    /**
-     * Create plan for reducers.
-     *
-     * @param top Topology.
-     * @param mappers Mappers map.
-     * @param reducerCnt Reducers count.
-     * @return Reducers map.
-     */
-    private Map<UUID, int[]> reducers(Collection<ClusterNode> top,
-        Map<UUID, Collection<GridHadoopInputSplit>> mappers, int reducerCnt) {
-        // Determine initial node weights.
-        int totalWeight = 0;
-
-        List<WeightedNode> nodes = new ArrayList<>(top.size());
-
-        for (ClusterNode node : top) {
-            Collection<GridHadoopInputSplit> split = mappers.get(node.id());
-
-            int weight = reducerNodeWeight(node, split != null ? split.size() : 0);
-
-            nodes.add(new WeightedNode(node.id(), weight, weight));
-
-            totalWeight += weight;
-        }
-
-        // Adjust weights.
-        int totalAdjustedWeight = 0;
-
-        for (WeightedNode node : nodes) {
-            node.floatWeight = ((float)node.weight * reducerCnt) / totalWeight;
-
-            node.weight = Math.round(node.floatWeight);
-
-            totalAdjustedWeight += node.weight;
-        }
-
-        // Apply redundant/lost reducers.
-        Collections.sort(nodes);
-
-        if (totalAdjustedWeight > reducerCnt) {
-            // Too much reducers set.
-            ListIterator<WeightedNode> iter = nodes.listIterator(nodes.size() - 1);
-
-            while (totalAdjustedWeight != reducerCnt) {
-                if (!iter.hasPrevious())
-                    iter = nodes.listIterator(nodes.size() - 1);
-
-                WeightedNode node = iter.previous();
-
-                if (node.weight > 0) {
-                    node.weight -= 1;
-
-                    totalAdjustedWeight--;
-                }
-            }
-        }
-        else if (totalAdjustedWeight < reducerCnt) {
-            // Not enough reducers set.
-            ListIterator<WeightedNode> iter = nodes.listIterator(0);
-
-            while (totalAdjustedWeight != reducerCnt) {
-                if (!iter.hasNext())
-                    iter = nodes.listIterator(0);
-
-                WeightedNode node = iter.next();
-
-                if (node.floatWeight > 0.0f) {
-                    node.weight += 1;
-
-                    totalAdjustedWeight++;
-                }
-            }
-        }
-
-        int idx = 0;
-
-        Map<UUID, int[]> reducers = new HashMap<>(nodes.size(), 1.0f);
-
-        for (WeightedNode node : nodes) {
-            if (node.weight > 0) {
-                int[] arr = new int[node.weight];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = idx++;
-
-                reducers.put(node.nodeId, arr);
-            }
-        }
-
-        return reducers;
-    }
-
-    /**
-     * Calculate node weight based on node metrics and data co-location.
-     *
-     * @param node Node.
-     * @param splitCnt Splits mapped to this node.
-     * @return Node weight.
-     */
-    @SuppressWarnings("UnusedParameters")
-    protected int reducerNodeWeight(ClusterNode node, int splitCnt) {
-        return splitCnt;
-    }
-
-    /**
-     * Weighted node.
-     */
-    private static class WeightedNode implements Comparable<WeightedNode> {
-        /** Node ID. */
-        private final UUID nodeId;
-
-        /** Weight. */
-        private int weight;
-
-        /** Floating point weight. */
-        private float floatWeight;
-
-        /**
-         * Constructor.
-         *
-         * @param nodeId Node ID.
-         * @param weight Weight.
-         * @param floatWeight Floating point weight.
-         */
-        private WeightedNode(UUID nodeId, int weight, float floatWeight) {
-            this.nodeId = nodeId;
-            this.weight = weight;
-            this.floatWeight = floatWeight;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj != null && obj instanceof WeightedNode && F.eq(nodeId, ((WeightedNode)obj).nodeId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return nodeId.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(@NotNull WeightedNode other) {
-            float res = other.floatWeight - floatWeight;
-
-            return res > 0.0f ? 1 : res < 0.0f ? -1 : nodeId.compareTo(other.nodeId);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
new file mode 100644
index 0000000..9ec2b5b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
@@ -0,0 +1,107 @@
+/*
+ * 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.planner;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Map-reduce plan.
+ */
+public class HadoopDefaultMapReducePlan implements GridHadoopMapReducePlan {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Mappers map. */
+    private Map<UUID, Collection<GridHadoopInputSplit>> mappers;
+
+    /** Reducers map. */
+    private Map<UUID, int[]> reducers;
+
+    /** Mappers count. */
+    private int mappersCnt;
+
+    /** Reducers count. */
+    private int reducersCnt;
+
+    /**
+     * @param mappers Mappers map.
+     * @param reducers Reducers map.
+     */
+    public HadoopDefaultMapReducePlan(Map<UUID, Collection<GridHadoopInputSplit>> mappers,
+        Map<UUID, int[]> reducers) {
+        this.mappers = mappers;
+        this.reducers = reducers;
+
+        if (mappers != null) {
+            for (Collection<GridHadoopInputSplit> splits : mappers.values())
+                mappersCnt += splits.size();
+        }
+
+        if (reducers != null) {
+            for (int[] rdcrs : reducers.values())
+                reducersCnt += rdcrs.length;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int mappers() {
+        return mappersCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int reducers() {
+        return reducersCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID nodeForReducer(int reducer) {
+        assert reducer >= 0 && reducer < reducersCnt : reducer;
+
+        for (Map.Entry<UUID, int[]> entry : reducers.entrySet()) {
+            for (int r : entry.getValue()) {
+                if (r == reducer)
+                    return entry.getKey();
+            }
+        }
+
+        throw new IllegalStateException("Not found reducer index: " + reducer);
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public Collection<GridHadoopInputSplit> mappers(UUID nodeId) {
+        return mappers.get(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override @Nullable public int[] reducers(UUID nodeId) {
+        return reducers.get(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<UUID> mapperNodeIds() {
+        return mappers.keySet();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<UUID> reducerNodeIds() {
+        return reducers.keySet();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
new file mode 100644
index 0000000..01a7471
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlanner.java
@@ -0,0 +1,434 @@
+/*
+ * 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.planner;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.igfs.hadoop.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+import static org.apache.ignite.IgniteFs.*;
+
+/**
+ * Default map-reduce planner implementation.
+ */
+public class HadoopDefaultMapReducePlanner implements GridHadoopMapReducePlanner {
+    /** Injected grid. */
+    @IgniteInstanceResource
+    private Ignite ignite;
+
+    /** Logger. */
+    @SuppressWarnings("UnusedDeclaration")
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopMapReducePlan preparePlan(GridHadoopJob job, Collection<ClusterNode> top,
+        @Nullable GridHadoopMapReducePlan oldPlan) throws IgniteCheckedException {
+        // Convert collection of topology nodes to collection of topology node IDs.
+        Collection<UUID> topIds = new HashSet<>(top.size(), 1.0f);
+
+        for (ClusterNode topNode : top)
+            topIds.add(topNode.id());
+
+        Map<UUID, Collection<GridHadoopInputSplit>> mappers = mappers(top, topIds, job.input());
+
+        int rdcCnt = job.info().reducers();
+
+        if (rdcCnt < 0)
+            throw new IgniteCheckedException("Number of reducers must be non-negative, actual: " + rdcCnt);
+
+        Map<UUID, int[]> reducers = reducers(top, mappers, rdcCnt);
+
+        return new HadoopDefaultMapReducePlan(mappers, reducers);
+    }
+
+    /**
+     * Create plan for mappers.
+     *
+     * @param top Topology nodes.
+     * @param topIds Topology node IDs.
+     * @param splits Splits.
+     * @return Mappers map.
+     * @throws IgniteCheckedException If failed.
+     */
+    private Map<UUID, Collection<GridHadoopInputSplit>> mappers(Collection<ClusterNode> top, Collection<UUID> topIds,
+        Iterable<GridHadoopInputSplit> splits) throws IgniteCheckedException {
+        Map<UUID, Collection<GridHadoopInputSplit>> mappers = new HashMap<>();
+
+        Map<String, Collection<UUID>> nodes = hosts(top);
+
+        Map<UUID, Integer> nodeLoads = new HashMap<>(top.size(), 1.0f); // Track node load.
+
+        for (UUID nodeId : topIds)
+            nodeLoads.put(nodeId, 0);
+
+        for (GridHadoopInputSplit split : splits) {
+            UUID nodeId = nodeForSplit(split, topIds, nodes, nodeLoads);
+
+            if (log.isDebugEnabled())
+                log.debug("Mapped split to node [split=" + split + ", nodeId=" + nodeId + ']');
+
+            Collection<GridHadoopInputSplit> nodeSplits = mappers.get(nodeId);
+
+            if (nodeSplits == null) {
+                nodeSplits = new ArrayList<>();
+
+                mappers.put(nodeId, nodeSplits);
+            }
+
+            nodeSplits.add(split);
+
+            // Updated node load.
+            nodeLoads.put(nodeId, nodeLoads.get(nodeId) + 1);
+        }
+
+        return mappers;
+    }
+
+    /**
+     * Groups nodes by host names.
+     *
+     * @param top Topology to group.
+     * @return Map.
+     */
+    private static Map<String, Collection<UUID>> hosts(Collection<ClusterNode> top) {
+        Map<String, Collection<UUID>> grouped = U.newHashMap(top.size());
+
+        for (ClusterNode node : top) {
+            for (String host : node.hostNames()) {
+                Collection<UUID> nodeIds = grouped.get(host);
+
+                if (nodeIds == null) {
+                    // Expecting 1-2 nodes per host.
+                    nodeIds = new ArrayList<>(2);
+
+                    grouped.put(host, nodeIds);
+                }
+
+                nodeIds.add(node.id());
+            }
+        }
+
+        return grouped;
+    }
+
+    /**
+     * Determine the best node for this split.
+     *
+     * @param split Split.
+     * @param topIds Topology node IDs.
+     * @param nodes Nodes.
+     * @param nodeLoads Node load tracker.
+     * @return Node ID.
+     */
+    @SuppressWarnings("unchecked")
+    private UUID nodeForSplit(GridHadoopInputSplit split, Collection<UUID> topIds, Map<String, Collection<UUID>> nodes,
+        Map<UUID, Integer> nodeLoads) throws IgniteCheckedException {
+        if (split instanceof GridHadoopFileBlock) {
+            GridHadoopFileBlock split0 = (GridHadoopFileBlock)split;
+
+            if (IGFS_SCHEME.equalsIgnoreCase(split0.file().getScheme())) {
+                IgfsHadoopEndpoint endpoint = new IgfsHadoopEndpoint(split0.file().getAuthority());
+
+                IgfsEx igfs = null;
+
+                if (F.eq(ignite.name(), endpoint.grid()))
+                    igfs = (IgfsEx)((IgniteEx)ignite).igfsx(endpoint.igfs());
+
+                if (igfs != null && !igfs.isProxy(split0.file())) {
+                    Collection<IgfsBlockLocation> blocks;
+
+                    try {
+                        blocks = igfs.affinity(new IgfsPath(split0.file()), split0.start(), split0.length());
+                    }
+                    catch (IgniteException e) {
+                        throw new IgniteCheckedException(e);
+                    }
+
+                    assert blocks != null;
+
+                    if (blocks.size() == 1)
+                        // Fast-path, split consists of one IGFS block (as in most cases).
+                        return bestNode(blocks.iterator().next().nodeIds(), topIds, nodeLoads, false);
+                    else {
+                        // Slow-path, file consists of multiple IGFS blocks. First, find the most co-located nodes.
+                        Map<UUID, Long> nodeMap = new HashMap<>();
+
+                        List<UUID> bestNodeIds = null;
+                        long bestLen = -1L;
+
+                        for (IgfsBlockLocation block : blocks) {
+                            for (UUID blockNodeId : block.nodeIds()) {
+                                if (topIds.contains(blockNodeId)) {
+                                    Long oldLen = nodeMap.get(blockNodeId);
+                                    long newLen = oldLen == null ? block.length() : oldLen + block.length();
+
+                                    nodeMap.put(blockNodeId, newLen);
+
+                                    if (bestNodeIds == null || bestLen < newLen) {
+                                        bestNodeIds = new ArrayList<>(1);
+
+                                        bestNodeIds.add(blockNodeId);
+
+                                        bestLen = newLen;
+                                    }
+                                    else if (bestLen == newLen) {
+                                        assert !F.isEmpty(bestNodeIds);
+
+                                        bestNodeIds.add(blockNodeId);
+                                    }
+                                }
+                            }
+                        }
+
+                        if (bestNodeIds != null) {
+                            return bestNodeIds.size() == 1 ? bestNodeIds.get(0) :
+                                bestNode(bestNodeIds, topIds, nodeLoads, true);
+                        }
+                    }
+                }
+            }
+        }
+
+        // Cannot use local IGFS for some reason, try selecting the node by host.
+        Collection<UUID> blockNodes = null;
+
+        for (String host : split.hosts()) {
+            Collection<UUID> hostNodes = nodes.get(host);
+
+            if (!F.isEmpty(hostNodes)) {
+                if (blockNodes == null)
+                    blockNodes = new ArrayList<>(hostNodes);
+                else
+                    blockNodes.addAll(hostNodes);
+            }
+        }
+
+        return bestNode(blockNodes, topIds, nodeLoads, false);
+    }
+
+    /**
+     * Finds the best (the least loaded) node among the candidates.
+     *
+     * @param candidates Candidates.
+     * @param topIds Topology node IDs.
+     * @param nodeLoads Known node loads.
+     * @param skipTopCheck Whether to skip topology check.
+     * @return The best node.
+     */
+    private UUID bestNode(@Nullable Collection<UUID> candidates, Collection<UUID> topIds, Map<UUID, Integer> nodeLoads,
+        boolean skipTopCheck) {
+        UUID bestNode = null;
+        int bestLoad = Integer.MAX_VALUE;
+
+        if (candidates != null) {
+            for (UUID candidate : candidates) {
+                if (skipTopCheck || topIds.contains(candidate)) {
+                    int load = nodeLoads.get(candidate);
+
+                    if (bestNode == null || bestLoad > load) {
+                        bestNode = candidate;
+                        bestLoad = load;
+
+                        if (bestLoad == 0)
+                            break; // Minimum load possible, no need for further iterations.
+                    }
+                }
+            }
+        }
+
+        if (bestNode == null) {
+            // Blocks are located on nodes which are not Hadoop-enabled, assign to the least loaded one.
+            bestLoad = Integer.MAX_VALUE;
+
+            for (UUID nodeId : topIds) {
+                int load = nodeLoads.get(nodeId);
+
+                if (bestNode == null || bestLoad > load) {
+                    bestNode = nodeId;
+                    bestLoad = load;
+
+                    if (bestLoad == 0)
+                        break; // Minimum load possible, no need for further iterations.
+                }
+            }
+        }
+
+        assert bestNode != null;
+
+        return bestNode;
+    }
+
+    /**
+     * Create plan for reducers.
+     *
+     * @param top Topology.
+     * @param mappers Mappers map.
+     * @param reducerCnt Reducers count.
+     * @return Reducers map.
+     */
+    private Map<UUID, int[]> reducers(Collection<ClusterNode> top,
+        Map<UUID, Collection<GridHadoopInputSplit>> mappers, int reducerCnt) {
+        // Determine initial node weights.
+        int totalWeight = 0;
+
+        List<WeightedNode> nodes = new ArrayList<>(top.size());
+
+        for (ClusterNode node : top) {
+            Collection<GridHadoopInputSplit> split = mappers.get(node.id());
+
+            int weight = reducerNodeWeight(node, split != null ? split.size() : 0);
+
+            nodes.add(new WeightedNode(node.id(), weight, weight));
+
+            totalWeight += weight;
+        }
+
+        // Adjust weights.
+        int totalAdjustedWeight = 0;
+
+        for (WeightedNode node : nodes) {
+            node.floatWeight = ((float)node.weight * reducerCnt) / totalWeight;
+
+            node.weight = Math.round(node.floatWeight);
+
+            totalAdjustedWeight += node.weight;
+        }
+
+        // Apply redundant/lost reducers.
+        Collections.sort(nodes);
+
+        if (totalAdjustedWeight > reducerCnt) {
+            // Too much reducers set.
+            ListIterator<WeightedNode> iter = nodes.listIterator(nodes.size() - 1);
+
+            while (totalAdjustedWeight != reducerCnt) {
+                if (!iter.hasPrevious())
+                    iter = nodes.listIterator(nodes.size() - 1);
+
+                WeightedNode node = iter.previous();
+
+                if (node.weight > 0) {
+                    node.weight -= 1;
+
+                    totalAdjustedWeight--;
+                }
+            }
+        }
+        else if (totalAdjustedWeight < reducerCnt) {
+            // Not enough reducers set.
+            ListIterator<WeightedNode> iter = nodes.listIterator(0);
+
+            while (totalAdjustedWeight != reducerCnt) {
+                if (!iter.hasNext())
+                    iter = nodes.listIterator(0);
+
+                WeightedNode node = iter.next();
+
+                if (node.floatWeight > 0.0f) {
+                    node.weight += 1;
+
+                    totalAdjustedWeight++;
+                }
+            }
+        }
+
+        int idx = 0;
+
+        Map<UUID, int[]> reducers = new HashMap<>(nodes.size(), 1.0f);
+
+        for (WeightedNode node : nodes) {
+            if (node.weight > 0) {
+                int[] arr = new int[node.weight];
+
+                for (int i = 0; i < arr.length; i++)
+                    arr[i] = idx++;
+
+                reducers.put(node.nodeId, arr);
+            }
+        }
+
+        return reducers;
+    }
+
+    /**
+     * Calculate node weight based on node metrics and data co-location.
+     *
+     * @param node Node.
+     * @param splitCnt Splits mapped to this node.
+     * @return Node weight.
+     */
+    @SuppressWarnings("UnusedParameters")
+    protected int reducerNodeWeight(ClusterNode node, int splitCnt) {
+        return splitCnt;
+    }
+
+    /**
+     * Weighted node.
+     */
+    private static class WeightedNode implements Comparable<WeightedNode> {
+        /** Node ID. */
+        private final UUID nodeId;
+
+        /** Weight. */
+        private int weight;
+
+        /** Floating point weight. */
+        private float floatWeight;
+
+        /**
+         * Constructor.
+         *
+         * @param nodeId Node ID.
+         * @param weight Weight.
+         * @param floatWeight Floating point weight.
+         */
+        private WeightedNode(UUID nodeId, int weight, float floatWeight) {
+            this.nodeId = nodeId;
+            this.weight = weight;
+            this.floatWeight = floatWeight;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            return obj != null && obj instanceof WeightedNode && F.eq(nodeId, ((WeightedNode)obj).nodeId);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return nodeId.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int compareTo(@NotNull WeightedNode other) {
+            float res = other.floatWeight - floatWeight;
+
+            return res > 0.0f ? 1 : res < 0.0f ? -1 : nodeId.compareTo(other.nodeId);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobCountersTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobCountersTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobCountersTask.java
deleted file mode 100644
index 37073d9..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobCountersTask.java
+++ /dev/null
@@ -1,45 +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.proto;
-
-import org.apache.ignite.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-import java.util.*;
-
-/**
- * Task to get job counters.
- */
-public class GridHadoopProtocolJobCountersTask extends GridHadoopProtocolTaskAdapter<GridHadoopCounters> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopCounters run(ComputeJobContext jobCtx, GridHadoop hadoop,
-        GridHadoopProtocolTaskArguments args) throws IgniteCheckedException {
-
-        UUID nodeId = UUID.fromString(args.<String>get(0));
-        Integer id = args.get(1);
-
-        assert nodeId != null;
-        assert id != null;
-
-        return hadoop.counters(new GridHadoopJobId(nodeId, id));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
deleted file mode 100644
index de4f89c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
+++ /dev/null
@@ -1,81 +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.proto;
-
-import org.apache.ignite.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-
-import java.util.*;
-
-/**
- * Job status task.
- */
-public class GridHadoopProtocolJobStatusTask extends GridHadoopProtocolTaskAdapter<GridHadoopJobStatus> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Default poll delay */
-    private static final long DFLT_POLL_DELAY = 100L;
-
-    /** Attribute for held status. */
-    private static final String ATTR_HELD = "held";
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobStatus run(final ComputeJobContext jobCtx, GridHadoop hadoop,
-        GridHadoopProtocolTaskArguments args) throws IgniteCheckedException {
-        UUID nodeId = UUID.fromString(args.<String>get(0));
-        Integer id = args.get(1);
-        Long pollDelay = args.get(2);
-
-        assert nodeId != null;
-        assert id != null;
-
-        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
-
-        if (pollDelay == null)
-            pollDelay = DFLT_POLL_DELAY;
-
-        if (pollDelay > 0) {
-            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
-
-            if (fut != null) {
-                if (fut.isDone() || F.eq(jobCtx.getAttribute(ATTR_HELD), true))
-                    return hadoop.status(jobId);
-                else {
-                    fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
-                        @Override public void apply(IgniteInternalFuture<?> fut0) {
-                            jobCtx.callcc();
-                        }
-                    });
-
-                    jobCtx.setAttribute(ATTR_HELD, true);
-
-                    return jobCtx.holdcc(pollDelay);
-                }
-            }
-            else
-                return null;
-        }
-        else
-            return hadoop.status(jobId);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolKillJobTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolKillJobTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolKillJobTask.java
deleted file mode 100644
index 384bc23..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolKillJobTask.java
+++ /dev/null
@@ -1,46 +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.proto;
-
-import org.apache.ignite.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-import java.util.*;
-
-/**
- * Kill job task.
- */
-public class GridHadoopProtocolKillJobTask extends GridHadoopProtocolTaskAdapter<Boolean> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public Boolean run(ComputeJobContext jobCtx, GridHadoop hadoop,
-        GridHadoopProtocolTaskArguments args) throws IgniteCheckedException {
-        UUID nodeId = UUID.fromString(args.<String>get(0));
-        Integer id = args.get(1);
-
-        assert nodeId != null;
-        assert id != null;
-
-        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
-
-        return hadoop.kill(jobId);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolNextTaskIdTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolNextTaskIdTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolNextTaskIdTask.java
deleted file mode 100644
index f76f3b6..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolNextTaskIdTask.java
+++ /dev/null
@@ -1,35 +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.proto;
-
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-/**
- * Task to get the next job ID.
- */
-public class GridHadoopProtocolNextTaskIdTask extends GridHadoopProtocolTaskAdapter<GridHadoopJobId> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobId run(ComputeJobContext jobCtx, GridHadoop hadoop,
-        GridHadoopProtocolTaskArguments args) {
-        return hadoop.nextJobId();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
deleted file mode 100644
index 8fdab9d..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolSubmitJobTask.java
+++ /dev/null
@@ -1,57 +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.proto;
-
-import org.apache.ignite.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-
-import java.util.*;
-
-import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
-
-/**
- * Submit job task.
- */
-public class GridHadoopProtocolSubmitJobTask extends GridHadoopProtocolTaskAdapter<GridHadoopJobStatus> {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public GridHadoopJobStatus run(ComputeJobContext jobCtx, GridHadoop hadoop,
-        GridHadoopProtocolTaskArguments args) throws IgniteCheckedException {
-        UUID nodeId = UUID.fromString(args.<String>get(0));
-        Integer id = args.get(1);
-        HadoopDefaultJobInfo info = args.get(2);
-
-        assert nodeId != null;
-        assert id != null;
-        assert info != null;
-
-        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
-
-        hadoop.submit(jobId, info);
-
-        GridHadoopJobStatus res = hadoop.status(jobId);
-
-        if (res == null) // Submission failed.
-            res = new GridHadoopJobStatus(jobId, info.jobName(), info.user(), 0, 0, 0, 0, PHASE_CANCELLING, true, 1);
-
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskAdapter.java
deleted file mode 100644
index 086545c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskAdapter.java
+++ /dev/null
@@ -1,113 +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.proto;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.resources.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Hadoop protocol task adapter.
- */
-public abstract class GridHadoopProtocolTaskAdapter<R> implements ComputeTask<GridHadoopProtocolTaskArguments, R> {
-    /** {@inheritDoc} */
-    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
-        @Nullable GridHadoopProtocolTaskArguments arg) {
-        return Collections.singletonMap(new Job(arg), subgrid.get(0));
-    }
-
-    /** {@inheritDoc} */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
-        return ComputeJobResultPolicy.REDUCE;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public R reduce(List<ComputeJobResult> results) {
-        if (!F.isEmpty(results)) {
-            ComputeJobResult res = results.get(0);
-
-            return res.getData();
-        }
-        else
-            return null;
-    }
-
-    /**
-     * Job wrapper.
-     */
-    private class Job implements ComputeJob {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /** */
-        @SuppressWarnings("UnusedDeclaration")
-        @JobContextResource
-        private ComputeJobContext jobCtx;
-
-        /** Argument. */
-        private final GridHadoopProtocolTaskArguments args;
-
-        /**
-         * Constructor.
-         *
-         * @param args Job argument.
-         */
-        private Job(GridHadoopProtocolTaskArguments args) {
-            this.args = args;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void cancel() {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
-        @Nullable @Override public Object execute() {
-            try {
-                return run(jobCtx, ((IgniteEx)ignite).hadoop(), args);
-            }
-            catch (IgniteCheckedException e) {
-                throw U.convertException(e);
-            }
-        }
-    }
-
-    /**
-     * Run the task.
-     *
-     * @param jobCtx Job context.
-     * @param hadoop Hadoop facade.
-     * @param args Arguments.
-     * @return Job result.
-     * @throws IgniteCheckedException If failed.
-     */
-    public abstract R run(ComputeJobContext jobCtx, GridHadoop hadoop, GridHadoopProtocolTaskArguments args)
-        throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskArguments.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskArguments.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskArguments.java
deleted file mode 100644
index ae91a52..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolTaskArguments.java
+++ /dev/null
@@ -1,81 +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.proto;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-/**
- * Task arguments.
- */
-public class GridHadoopProtocolTaskArguments implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Arguments. */
-    private Object[] args;
-
-    /**
-     * {@link Externalizable} support.
-     */
-    public GridHadoopProtocolTaskArguments() {
-        // No-op.
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param args Arguments.
-     */
-    public GridHadoopProtocolTaskArguments(Object... args) {
-        this.args = args;
-    }
-
-    /**
-     * @param idx Argument index.
-     * @return Argument.
-     */
-    @SuppressWarnings("unchecked")
-    @Nullable public <T> T get(int idx) {
-        return (args != null && args.length > idx) ? (T)args[idx] : null;
-    }
-
-    /**
-     * @return Size.
-     */
-    public int size() {
-        return args != null ? args.length : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        U.writeArray(out, args);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        args = U.readArray(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopProtocolTaskArguments.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
index 66fb230..3a766c3 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopClientProtocol.java
@@ -82,7 +82,7 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             conf.setLong(REQ_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
 
-            GridHadoopJobId jobID = cli.compute().execute(GridHadoopProtocolNextTaskIdTask.class.getName(), null);
+            GridHadoopJobId jobID = cli.compute().execute(HadoopProtocolNextTaskIdTask.class.getName(), null);
 
             conf.setLong(RESPONSE_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
 
@@ -99,8 +99,8 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             conf.setLong(JOB_SUBMISSION_START_TS_PROPERTY, U.currentTimeMillis());
 
-            GridHadoopJobStatus status = cli.compute().execute(GridHadoopProtocolSubmitJobTask.class.getName(),
-                new GridHadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), createJobInfo(conf)));
+            GridHadoopJobStatus status = cli.compute().execute(HadoopProtocolSubmitJobTask.class.getName(),
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), createJobInfo(conf)));
 
             if (status == null)
                 throw new IOException("Failed to submit job (null status obtained): " + jobId);
@@ -135,8 +135,8 @@ public class HadoopClientProtocol implements ClientProtocol {
     /** {@inheritDoc} */
     @Override public void killJob(JobID jobId) throws IOException, InterruptedException {
         try {
-            cli.compute().execute(GridHadoopProtocolKillJobTask.class.getName(),
-                new GridHadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
+            cli.compute().execute(HadoopProtocolKillJobTask.class.getName(),
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
         }
         catch (GridClientException e) {
             throw new IOException("Failed to kill job: " + jobId, e);
@@ -159,11 +159,11 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             Long delay = conf.getLong(GridHadoopJobProperty.JOB_STATUS_POLL_DELAY.propertyName(), -1);
 
-            GridHadoopProtocolTaskArguments args = delay >= 0 ?
-                new GridHadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), delay) :
-                new GridHadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId());
+            HadoopProtocolTaskArguments args = delay >= 0 ?
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), delay) :
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId());
 
-            GridHadoopJobStatus status = cli.compute().execute(GridHadoopProtocolJobStatusTask.class.getName(), args);
+            GridHadoopJobStatus status = cli.compute().execute(HadoopProtocolJobStatusTask.class.getName(), args);
 
             if (status == null)
                 throw new IOException("Job tracker doesn't have any information about the job: " + jobId);
@@ -178,8 +178,8 @@ public class HadoopClientProtocol implements ClientProtocol {
     /** {@inheritDoc} */
     @Override public Counters getJobCounters(JobID jobId) throws IOException, InterruptedException {
         try {
-            final GridHadoopCounters counters = cli.compute().execute(GridHadoopProtocolJobCountersTask.class.getName(),
-                new GridHadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
+            final GridHadoopCounters counters = cli.compute().execute(HadoopProtocolJobCountersTask.class.getName(),
+                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
 
             if (counters == null)
                 throw new IOException("Job tracker doesn't have any information about the job: " + jobId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobCountersTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobCountersTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobCountersTask.java
new file mode 100644
index 0000000..56da194
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobCountersTask.java
@@ -0,0 +1,45 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+import java.util.*;
+
+/**
+ * Task to get job counters.
+ */
+public class HadoopProtocolJobCountersTask extends HadoopProtocolTaskAdapter<GridHadoopCounters> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopCounters run(ComputeJobContext jobCtx, GridHadoop hadoop,
+        HadoopProtocolTaskArguments args) throws IgniteCheckedException {
+
+        UUID nodeId = UUID.fromString(args.<String>get(0));
+        Integer id = args.get(1);
+
+        assert nodeId != null;
+        assert id != null;
+
+        return hadoop.counters(new GridHadoopJobId(nodeId, id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobStatusTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobStatusTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobStatusTask.java
new file mode 100644
index 0000000..ac70c44
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolJobStatusTask.java
@@ -0,0 +1,81 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+
+/**
+ * Job status task.
+ */
+public class HadoopProtocolJobStatusTask extends HadoopProtocolTaskAdapter<GridHadoopJobStatus> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Default poll delay */
+    private static final long DFLT_POLL_DELAY = 100L;
+
+    /** Attribute for held status. */
+    private static final String ATTR_HELD = "held";
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobStatus run(final ComputeJobContext jobCtx, GridHadoop hadoop,
+        HadoopProtocolTaskArguments args) throws IgniteCheckedException {
+        UUID nodeId = UUID.fromString(args.<String>get(0));
+        Integer id = args.get(1);
+        Long pollDelay = args.get(2);
+
+        assert nodeId != null;
+        assert id != null;
+
+        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
+
+        if (pollDelay == null)
+            pollDelay = DFLT_POLL_DELAY;
+
+        if (pollDelay > 0) {
+            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
+
+            if (fut != null) {
+                if (fut.isDone() || F.eq(jobCtx.getAttribute(ATTR_HELD), true))
+                    return hadoop.status(jobId);
+                else {
+                    fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut0) {
+                            jobCtx.callcc();
+                        }
+                    });
+
+                    jobCtx.setAttribute(ATTR_HELD, true);
+
+                    return jobCtx.holdcc(pollDelay);
+                }
+            }
+            else
+                return null;
+        }
+        else
+            return hadoop.status(jobId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolKillJobTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolKillJobTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolKillJobTask.java
new file mode 100644
index 0000000..8522ab0
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolKillJobTask.java
@@ -0,0 +1,46 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+import java.util.*;
+
+/**
+ * Kill job task.
+ */
+public class HadoopProtocolKillJobTask extends HadoopProtocolTaskAdapter<Boolean> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public Boolean run(ComputeJobContext jobCtx, GridHadoop hadoop,
+        HadoopProtocolTaskArguments args) throws IgniteCheckedException {
+        UUID nodeId = UUID.fromString(args.<String>get(0));
+        Integer id = args.get(1);
+
+        assert nodeId != null;
+        assert id != null;
+
+        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
+
+        return hadoop.kill(jobId);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolNextTaskIdTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolNextTaskIdTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolNextTaskIdTask.java
new file mode 100644
index 0000000..357e12d
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolNextTaskIdTask.java
@@ -0,0 +1,35 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+/**
+ * Task to get the next job ID.
+ */
+public class HadoopProtocolNextTaskIdTask extends HadoopProtocolTaskAdapter<GridHadoopJobId> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobId run(ComputeJobContext jobCtx, GridHadoop hadoop,
+        HadoopProtocolTaskArguments args) {
+        return hadoop.nextJobId();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolSubmitJobTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolSubmitJobTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolSubmitJobTask.java
new file mode 100644
index 0000000..df03c79
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolSubmitJobTask.java
@@ -0,0 +1,57 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+
+import java.util.*;
+
+import static org.apache.ignite.internal.processors.hadoop.GridHadoopJobPhase.*;
+
+/**
+ * Submit job task.
+ */
+public class HadoopProtocolSubmitJobTask extends HadoopProtocolTaskAdapter<GridHadoopJobStatus> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public GridHadoopJobStatus run(ComputeJobContext jobCtx, GridHadoop hadoop,
+        HadoopProtocolTaskArguments args) throws IgniteCheckedException {
+        UUID nodeId = UUID.fromString(args.<String>get(0));
+        Integer id = args.get(1);
+        HadoopDefaultJobInfo info = args.get(2);
+
+        assert nodeId != null;
+        assert id != null;
+        assert info != null;
+
+        GridHadoopJobId jobId = new GridHadoopJobId(nodeId, id);
+
+        hadoop.submit(jobId, info);
+
+        GridHadoopJobStatus res = hadoop.status(jobId);
+
+        if (res == null) // Submission failed.
+            res = new GridHadoopJobStatus(jobId, info.jobName(), info.user(), 0, 0, 0, 0, PHASE_CANCELLING, true, 1);
+
+        return res;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskAdapter.java
new file mode 100644
index 0000000..6938d1c
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskAdapter.java
@@ -0,0 +1,113 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Hadoop protocol task adapter.
+ */
+public abstract class HadoopProtocolTaskAdapter<R> implements ComputeTask<HadoopProtocolTaskArguments, R> {
+    /** {@inheritDoc} */
+    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
+        @Nullable HadoopProtocolTaskArguments arg) {
+        return Collections.singletonMap(new Job(arg), subgrid.get(0));
+    }
+
+    /** {@inheritDoc} */
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) {
+        return ComputeJobResultPolicy.REDUCE;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public R reduce(List<ComputeJobResult> results) {
+        if (!F.isEmpty(results)) {
+            ComputeJobResult res = results.get(0);
+
+            return res.getData();
+        }
+        else
+            return null;
+    }
+
+    /**
+     * Job wrapper.
+     */
+    private class Job implements ComputeJob {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** */
+        @SuppressWarnings("UnusedDeclaration")
+        @JobContextResource
+        private ComputeJobContext jobCtx;
+
+        /** Argument. */
+        private final HadoopProtocolTaskArguments args;
+
+        /**
+         * Constructor.
+         *
+         * @param args Job argument.
+         */
+        private Job(HadoopProtocolTaskArguments args) {
+            this.args = args;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public Object execute() {
+            try {
+                return run(jobCtx, ((IgniteEx)ignite).hadoop(), args);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+        }
+    }
+
+    /**
+     * Run the task.
+     *
+     * @param jobCtx Job context.
+     * @param hadoop Hadoop facade.
+     * @param args Arguments.
+     * @return Job result.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract R run(ComputeJobContext jobCtx, GridHadoop hadoop, HadoopProtocolTaskArguments args)
+        throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskArguments.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskArguments.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskArguments.java
new file mode 100644
index 0000000..5c470ba
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/HadoopProtocolTaskArguments.java
@@ -0,0 +1,81 @@
+/*
+ * 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.proto;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Task arguments.
+ */
+public class HadoopProtocolTaskArguments implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Arguments. */
+    private Object[] args;
+
+    /**
+     * {@link Externalizable} support.
+     */
+    public HadoopProtocolTaskArguments() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param args Arguments.
+     */
+    public HadoopProtocolTaskArguments(Object... args) {
+        this.args = args;
+    }
+
+    /**
+     * @param idx Argument index.
+     * @return Argument.
+     */
+    @SuppressWarnings("unchecked")
+    @Nullable public <T> T get(int idx) {
+        return (args != null && args.length > idx) ? (T)args[idx] : null;
+    }
+
+    /**
+     * @return Size.
+     */
+    public int size() {
+        return args != null ? args.length : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        U.writeArray(out, args);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        args = U.readArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopProtocolTaskArguments.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleAck.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleAck.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleAck.java
deleted file mode 100644
index a8a52a9..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleAck.java
+++ /dev/null
@@ -1,91 +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.shuffle;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Acknowledgement message.
- */
-public class GridHadoopShuffleAck implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    @GridToStringInclude
-    private long msgId;
-
-    /** */
-    @GridToStringInclude
-    private GridHadoopJobId jobId;
-
-    /**
-     *
-     */
-    public GridHadoopShuffleAck() {
-        // No-op.
-    }
-
-    /**
-     * @param msgId Message ID.
-     */
-    public GridHadoopShuffleAck(long msgId, GridHadoopJobId jobId) {
-        assert jobId != null;
-
-        this.msgId = msgId;
-        this.jobId = jobId;
-    }
-
-    /**
-     * @return Message ID.
-     */
-    public long id() {
-        return msgId;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        jobId.writeExternal(out);
-        out.writeLong(msgId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = new GridHadoopJobId();
-
-        jobId.readExternal(in);
-        msgId = in.readLong();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopShuffleAck.class, this);
-    }
-}


[24/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (4).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java
new file mode 100644
index 0000000..2be65fd
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemLoopbackPrimarySelfTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs;
+
+import java.util.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
+
+/**
+ * Tests Hadoop 2.x file system in primary mode.
+ */
+public class HadoopIgfs20FileSystemLoopbackPrimarySelfTest extends HadoopIgfs20FileSystemAbstractSelfTest {
+    /**
+     * Creates test in primary mode.
+     */
+    public HadoopIgfs20FileSystemLoopbackPrimarySelfTest() {
+        super(PRIMARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String primaryFileSystemUriPath() {
+        return "igfs://igfs:" + getTestGridName(0) + "@/";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String primaryFileSystemConfigPath() {
+        return "/modules/core/src/test/config/hadoop/core-site-loopback.xml";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Map<String, String> primaryIpcEndpointConfiguration(final String gridName) {
+        return new HashMap<String, String>() {{
+            put("type", "tcp");
+            put("port", String.valueOf(DFLT_IPC_PORT + getTestGridIndex(gridName)));
+        }};
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String secondaryFileSystemUriPath() {
+        assert false;
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String secondaryFileSystemConfigPath() {
+        assert false;
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Map<String, String> secondaryIpcEndpointConfiguration() {
+        assert false;
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java
new file mode 100644
index 0000000..93f2d4a
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemShmemPrimarySelfTest.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs;
+
+import java.util.*;
+
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.util.ipc.shmem.IpcSharedMemoryServerEndpoint.*;
+
+/**
+ * Tests Hadoop 2.x file system in primary mode.
+ */
+public class HadoopIgfs20FileSystemShmemPrimarySelfTest extends HadoopIgfs20FileSystemAbstractSelfTest {
+    /**
+     * Creates test in primary mode.
+     */
+    public HadoopIgfs20FileSystemShmemPrimarySelfTest() {
+        super(PRIMARY);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String primaryFileSystemUriPath() {
+        return "igfs://igfs:" + getTestGridName(0) + "@/";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String primaryFileSystemConfigPath() {
+        return "/modules/core/src/test/config/hadoop/core-site.xml";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Map<String, String> primaryIpcEndpointConfiguration(final String gridName) {
+        return new HashMap<String, String>() {{
+            put("type", "shmem");
+            put("port", String.valueOf(DFLT_IPC_PORT + getTestGridIndex(gridName)));
+        }};
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String secondaryFileSystemUriPath() {
+        assert false;
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected String secondaryFileSystemConfigPath() {
+        assert false;
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Map<String, String> secondaryIpcEndpointConfiguration() {
+        assert false;
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
new file mode 100644
index 0000000..be7bb6d
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfsDualAbstractSelfTest.java
@@ -0,0 +1,304 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.igfs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.hadoop.fs.*;
+import org.apache.ignite.internal.processors.igfs.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
+import static org.apache.ignite.internal.processors.igfs.IgfsAbstractSelfTest.*;
+
+/**
+ * Tests for IGFS working in mode when remote file system exists: DUAL_SYNC, DUAL_ASYNC.
+ */
+public abstract class HadoopIgfsDualAbstractSelfTest extends IgfsCommonAbstractTest {
+    /** IGFS block size. */
+    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
+
+    /** Amount of blocks to prefetch. */
+    protected static final int PREFETCH_BLOCKS = 1;
+
+    /** Amount of sequential block reads before prefetch is triggered. */
+    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
+
+    /** Secondary file system URI. */
+    protected static final String SECONDARY_URI = "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/";
+
+    /** Secondary file system configuration path. */
+    protected static final String SECONDARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml";
+
+    /** Primary file system URI. */
+    protected static final String PRIMARY_URI = "igfs://igfs:grid@/";
+
+    /** Primary file system configuration path. */
+    protected static final String PRIMARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback.xml";
+
+    /** Primary file system REST endpoint configuration map. */
+    protected static final Map<String, String> PRIMARY_REST_CFG = new HashMap<String, String>() {{
+        put("type", "tcp");
+        put("port", "10500");
+    }};
+
+    /** Secondary file system REST endpoint configuration map. */
+    protected static final Map<String, String> SECONDARY_REST_CFG = new HashMap<String, String>() {{
+        put("type", "tcp");
+        put("port", "11500");
+    }};
+
+    /** Directory. */
+    protected static final IgfsPath DIR = new IgfsPath("/dir");
+
+    /** Sub-directory. */
+    protected static final IgfsPath SUBDIR = new IgfsPath(DIR, "subdir");
+
+    /** File. */
+    protected static final IgfsPath FILE = new IgfsPath(SUBDIR, "file");
+
+    /** Default data chunk (128 bytes). */
+    protected static byte[] chunk;
+
+    /** Primary IGFS. */
+    protected static IgfsImpl igfs;
+
+    /** Secondary IGFS. */
+    protected static IgfsImpl igfsSecondary;
+
+    /** IGFS mode. */
+    protected final IgfsMode mode;
+
+    /**
+     * Constructor.
+     *
+     * @param mode IGFS mode.
+     */
+    protected HadoopIgfsDualAbstractSelfTest(IgfsMode mode) {
+        this.mode = mode;
+        assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
+    }
+
+    /**
+     * Start grid with IGFS.
+     *
+     * @param gridName Grid name.
+     * @param igfsName IGFS name
+     * @param mode IGFS mode.
+     * @param secondaryFs Secondary file system (optional).
+     * @param restCfg Rest configuration string (optional).
+     * @return Started grid instance.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
+        @Nullable Igfs secondaryFs, @Nullable Map<String, String> restCfg) throws Exception {
+        IgfsConfiguration igfsCfg = new IgfsConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(igfsName);
+        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setIpcEndpointConfiguration(restCfg);
+        igfsCfg.setSecondaryFileSystem(secondaryFs);
+        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
+        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setQueryIndexEnabled(false);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+        dataCacheCfg.setOffHeapMaxMemory(0);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setQueryIndexEnabled(false);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setIgfsConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        return G.start(cfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        chunk = new byte[128];
+
+        for (int i = 0; i < chunk.length; i++)
+            chunk[i] = (byte)i;
+
+        Ignite igniteSecondary = startGridWithIgfs("grid-secondary", "igfs-secondary", PRIMARY, null, SECONDARY_REST_CFG);
+
+        Igfs hadoopFs = new IgniteHadoopSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG);
+
+        Ignite ignite = startGridWithIgfs("grid", "igfs", mode, hadoopFs, PRIMARY_REST_CFG);
+
+        igfsSecondary = (IgfsImpl) igniteSecondary.fileSystem("igfs-secondary");
+        igfs = (IgfsImpl) ignite.fileSystem("igfs");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        clear(igfs);
+        clear(igfsSecondary);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        G.stopAll(true);
+    }
+
+    /**
+     * Convenient method to group paths.
+     *
+     * @param paths Paths to group.
+     * @return Paths as array.
+     */
+    protected IgfsPath[] paths(IgfsPath... paths) {
+        return paths;
+    }
+
+    /**
+     * Check how prefetch override works.
+     *
+     * @throws Exception IF failed.
+     */
+    public void testOpenPrefetchOverride() throws Exception {
+        create(igfsSecondary, paths(DIR, SUBDIR), paths(FILE));
+
+        // Write enough data to the secondary file system.
+        final int blockSize = IGFS_BLOCK_SIZE;
+
+        IgfsOutputStream out = igfsSecondary.append(FILE, false);
+
+        int totalWritten = 0;
+
+        while (totalWritten < blockSize * 2 + chunk.length) {
+            out.write(chunk);
+
+            totalWritten += chunk.length;
+        }
+
+        out.close();
+
+        awaitFileClose(igfsSecondary, FILE);
+
+        // Instantiate file system with overridden "seq reads before prefetch" property.
+        Configuration cfg = new Configuration();
+
+        cfg.addResource(U.resolveIgniteUrl(PRIMARY_CFG));
+
+        int seqReads = SEQ_READS_BEFORE_PREFETCH + 1;
+
+        cfg.setInt(String.format(PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, "igfs:grid@"), seqReads);
+
+        FileSystem fs = FileSystem.get(new URI(PRIMARY_URI), cfg);
+
+        // Read the first two blocks.
+        Path fsHome = new Path(PRIMARY_URI);
+        Path dir = new Path(fsHome, DIR.name());
+        Path subdir = new Path(dir, SUBDIR.name());
+        Path file = new Path(subdir, FILE.name());
+
+        FSDataInputStream fsIn = fs.open(file);
+
+        final byte[] readBuf = new byte[blockSize * 2];
+
+        fsIn.readFully(0, readBuf, 0, readBuf.length);
+
+        // Wait for a while for prefetch to finish (if any).
+        IgfsMetaManager meta = igfs.context().meta();
+
+        IgfsFileInfo info = meta.info(meta.fileId(FILE));
+
+        IgfsBlockKey key = new IgfsBlockKey(info.id(), info.affinityKey(), info.evictExclude(), 2);
+
+        GridCache<IgfsBlockKey, byte[]> dataCache = igfs.context().kernalContext().cache().cache(
+            igfs.configuration().getDataCacheName());
+
+        for (int i = 0; i < 10; i++) {
+            if (dataCache.containsKey(key))
+                break;
+            else
+                U.sleep(100);
+        }
+
+        fsIn.close();
+
+        // Remove the file from the secondary file system.
+        igfsSecondary.delete(FILE, false);
+
+        // Try reading the third block. Should fail.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                IgfsInputStream in0 = igfs.open(FILE);
+
+                in0.seek(blockSize * 2);
+
+                try {
+                    in0.read(readBuf);
+                }
+                finally {
+                    U.closeQuiet(in0);
+                }
+
+                return null;
+            }
+        }, IOException.class,
+            "Failed to read data due to secondary file system exception: /dir/subdir/file");
+    }
+}

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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/17c8d0d9/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemAbstractSelfTest.java
deleted file mode 100644
index 04f776e..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsHadoop20FileSystemAbstractSelfTest.java
+++ /dev/null
@@ -1,1967 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.igfs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.permission.*;
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.hadoop.fs.*;
-import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.communication.*;
-import org.apache.ignite.spi.communication.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.igfs.IgfsMode.*;
-
-/**
- * Hadoop 2.x compliant file system.
- */
-public abstract class IgfsHadoop20FileSystemAbstractSelfTest extends IgfsCommonAbstractTest {
-    /** Group size. */
-    public static final int GRP_SIZE = 128;
-
-    /** Thread count for multithreaded tests. */
-    private static final int THREAD_CNT = 8;
-
-    /** IP finder. */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** Barrier for multithreaded tests. */
-    private static CyclicBarrier barrier;
-
-    /** File system. */
-    private static AbstractFileSystem fs;
-
-    /** Default IGFS mode. */
-    protected IgfsMode mode;
-
-    /** Primary file system URI. */
-    protected URI primaryFsUri;
-
-    /** Primary file system configuration. */
-    protected Configuration primaryFsCfg;
-
-    /**
-     * Constructor.
-     *
-     * @param mode Default IGFS mode.
-     */
-    protected IgfsHadoop20FileSystemAbstractSelfTest(IgfsMode mode) {
-        this.mode = mode;
-    }
-
-    /**
-     * Gets primary file system URI path.
-     *
-     * @return Primary file system URI path.
-     */
-    protected abstract String primaryFileSystemUriPath();
-
-    /**
-     * Gets primary file system config path.
-     *
-     * @return Primary file system config path.
-     */
-    protected abstract String primaryFileSystemConfigPath();
-
-    /**
-     * Get primary IPC endpoint configuration.
-     *
-     * @param gridName Grid name.
-     * @return IPC primary endpoint configuration.
-     */
-    protected abstract Map<String, String>  primaryIpcEndpointConfiguration(String gridName);
-
-    /**
-     * Gets secondary file system URI path.
-     *
-     * @return Secondary file system URI path.
-     */
-    protected abstract String secondaryFileSystemUriPath();
-
-    /**
-     * Gets secondary file system config path.
-     *
-     * @return Secondary file system config path.
-     */
-    protected abstract String secondaryFileSystemConfigPath();
-
-    /**
-     * Get secondary IPC endpoint configuration.
-     *
-     * @return Secondary IPC endpoint configuration.
-     */
-    protected abstract Map<String, String>  secondaryIpcEndpointConfiguration();
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        startNodes();
-    }
-
-    /**
-     * Starts the nodes for this test.
-     *
-     * @throws Exception If failed.
-     */
-    private void startNodes() throws Exception {
-        if (mode != PRIMARY) {
-            // Start secondary IGFS.
-            IgfsConfiguration igfsCfg = new IgfsConfiguration();
-
-            igfsCfg.setDataCacheName("partitioned");
-            igfsCfg.setMetaCacheName("replicated");
-            igfsCfg.setName("igfs_secondary");
-            igfsCfg.setIpcEndpointConfiguration(secondaryIpcEndpointConfiguration());
-            igfsCfg.setManagementPort(-1);
-            igfsCfg.setBlockSize(512 * 1024);
-            igfsCfg.setPrefetchBlocks(1);
-
-            CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-            cacheCfg.setName("partitioned");
-            cacheCfg.setCacheMode(PARTITIONED);
-            cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-            cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-            cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(GRP_SIZE));
-            cacheCfg.setBackups(0);
-            cacheCfg.setQueryIndexEnabled(false);
-            cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-            CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-            metaCacheCfg.setName("replicated");
-            metaCacheCfg.setCacheMode(REPLICATED);
-            metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-            metaCacheCfg.setQueryIndexEnabled(false);
-            metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            cfg.setGridName("grid_secondary");
-
-            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-            discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-            cfg.setDiscoverySpi(discoSpi);
-            cfg.setCacheConfiguration(metaCacheCfg, cacheCfg);
-            cfg.setIgfsConfiguration(igfsCfg);
-            cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-            cfg.setLocalHost(U.getLocalHost().getHostAddress());
-            cfg.setCommunicationSpi(communicationSpi());
-
-            G.start(cfg);
-        }
-
-        startGrids(4);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getTestGridName() {
-        return "grid";
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
-        cfg.setCacheConfiguration(cacheConfiguration(gridName));
-        cfg.setIgfsConfiguration(igfsConfiguration(gridName));
-        cfg.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-        cfg.setLocalHost("127.0.0.1");
-        cfg.setCommunicationSpi(communicationSpi());
-
-        return cfg;
-    }
-
-    /**
-     * Gets cache configuration.
-     *
-     * @param gridName Grid name.
-     * @return Cache configuration.
-     */
-    protected CacheConfiguration[] cacheConfiguration(String gridName) {
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName("partitioned");
-        cacheCfg.setCacheMode(PARTITIONED);
-        cacheCfg.setDistributionMode(CacheDistributionMode.PARTITIONED_ONLY);
-        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(GRP_SIZE));
-        cacheCfg.setBackups(0);
-        cacheCfg.setQueryIndexEnabled(false);
-        cacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
-
-        metaCacheCfg.setName("replicated");
-        metaCacheCfg.setCacheMode(REPLICATED);
-        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        metaCacheCfg.setQueryIndexEnabled(false);
-        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
-
-        return new CacheConfiguration[] {metaCacheCfg, cacheCfg};
-    }
-
-    /**
-     * Gets IGFS configuration.
-     *
-     * @param gridName Grid name.
-     * @return IGFS configuration.
-     */
-    protected IgfsConfiguration igfsConfiguration(String gridName) throws IgniteCheckedException {
-        IgfsConfiguration cfg = new IgfsConfiguration();
-
-        cfg.setDataCacheName("partitioned");
-        cfg.setMetaCacheName("replicated");
-        cfg.setName("igfs");
-        cfg.setPrefetchBlocks(1);
-        cfg.setMaxSpaceSize(64 * 1024 * 1024);
-        cfg.setDefaultMode(mode);
-
-        if (mode != PRIMARY)
-            cfg.setSecondaryFileSystem(new IgniteHadoopSecondaryFileSystem(secondaryFileSystemUriPath(),
-                secondaryFileSystemConfigPath()));
-
-        cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
-        cfg.setManagementPort(-1);
-
-        cfg.setBlockSize(512 * 1024); // Together with group blocks mapper will yield 64M per node groups.
-
-        return cfg;
-    }
-
-    /** @return Communication SPI. */
-    private CommunicationSpi communicationSpi() {
-        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
-
-        commSpi.setSharedMemoryPort(-1);
-
-        return commSpi;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        G.stopAll(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        primaryFsUri = new URI(primaryFileSystemUriPath());
-
-        primaryFsCfg = new Configuration();
-
-        primaryFsCfg.addResource(U.resolveIgniteUrl(primaryFileSystemConfigPath()));
-
-        fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
-
-        barrier = new CyclicBarrier(THREAD_CNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        try {
-            fs.delete(new Path("/"), true);
-        }
-        catch (Exception ignore) {
-            // No-op.
-        }
-
-        U.closeQuiet((Closeable)fs);
-    }
-
-    /** @throws Exception If failed. */
-    public void testStatus() throws Exception {
-
-        try (FSDataOutputStream file = fs.create(new Path("/file1"), EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()))) {
-            file.write(new byte[1024 * 1024]);
-        }
-
-        FsStatus status = fs.getFsStatus();
-
-        assertEquals(4, grid(0).cluster().nodes().size());
-
-        long used = 0, max = 0;
-
-        for (int i = 0; i < 4; i++) {
-            IgniteFs igfs = grid(i).fileSystem("igfs");
-
-            IgfsMetrics metrics = igfs.metrics();
-
-            used += metrics.localSpaceSize();
-            max += metrics.maxSpaceSize();
-        }
-
-        assertEquals(used, status.getUsed());
-        assertEquals(max, status.getCapacity());
-    }
-
-    /** @throws Exception If failed. */
-    public void testTimes() throws Exception {
-        Path file = new Path("/file1");
-
-        long now = System.currentTimeMillis();
-
-        try (FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()))) {
-            os.write(new byte[1024 * 1024]);
-        }
-
-        FileStatus status = fs.getFileStatus(file);
-
-        assertTrue(status.getAccessTime() >= now);
-        assertTrue(status.getModificationTime() >= now);
-
-        long accessTime = now - 10 * 60 * 1000;
-        long modificationTime = now - 5 * 60 * 1000;
-
-        fs.setTimes(file, modificationTime, accessTime);
-
-        status = fs.getFileStatus(file);
-        assertEquals(accessTime, status.getAccessTime());
-        assertEquals(modificationTime, status.getModificationTime());
-
-        // Check listing is updated as well.
-        FileStatus[] files = fs.listStatus(new Path("/"));
-
-        assertEquals(1, files.length);
-
-        assertEquals(file.getName(), files[0].getPath().getName());
-        assertEquals(accessTime, files[0].getAccessTime());
-        assertEquals(modificationTime, files[0].getModificationTime());
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.setTimes(new Path("/unknownFile"), 0, 0);
-
-                return null;
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testCreateCheckParameters() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.create(null, EnumSet.noneOf(CreateFlag.class),
-                    Options.CreateOpts.perms(FsPermission.getDefault()));
-            }
-        }, NullPointerException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testCreateBase() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
-        Path file = new Path(dir, "someFile");
-
-        assertPathDoesNotExist(fs, file);
-
-        FsPermission fsPerm = new FsPermission((short)644);
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(fsPerm));
-
-        // Try to write something in file.
-        os.write("abc".getBytes());
-
-        os.close();
-
-        // Check file status.
-        FileStatus fileStatus = fs.getFileStatus(file);
-
-        assertFalse(fileStatus.isDirectory());
-        assertEquals(file, fileStatus.getPath());
-        assertEquals(fsPerm, fileStatus.getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    public void testCreateCheckOverwrite() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
-        final Path file = new Path(dir, "someFile");
-
-        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        out.close();
-
-        // Check intermediate directory permissions.
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir).getPermission());
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir.getParent()).getPermission());
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir.getParent().getParent()).getPermission());
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.create(file, EnumSet.noneOf(CreateFlag.class),
-                    Options.CreateOpts.perms(FsPermission.getDefault()));
-            }
-        }, PathExistsException.class, null);
-
-        // Overwrite should be successful.
-        FSDataOutputStream out1 = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        out1.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteIfNoSuchPath() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
-
-        assertPathDoesNotExist(fs, dir);
-
-        assertFalse(fs.delete(dir, true));
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteSuccessfulIfPathIsOpenedToRead() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "myFile");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        final int cnt = 5 * IgfsConfiguration.DFLT_BLOCK_SIZE; // Write 5 blocks.
-
-        for (int i = 0; i < cnt; i++)
-            os.writeInt(i);
-
-        os.close();
-
-        final FSDataInputStream is = fs.open(file, -1);
-
-        for (int i = 0; i < cnt / 2; i++)
-            assertEquals(i, is.readInt());
-
-        assert fs.delete(file, false);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.getFileStatus(file);
-
-                return null;
-            }
-        }, FileNotFoundException.class, null);
-
-        is.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteIfFilePathExists() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "myFile");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        assertTrue(fs.delete(file, false));
-
-        assertPathDoesNotExist(fs, file);
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteIfDirectoryPathExists() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path dir = new Path(fsHome, "/someDir1/someDir2/someDir3");
-
-        FSDataOutputStream os = fs.create(dir, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        assertTrue(fs.delete(dir, false));
-
-        assertPathDoesNotExist(fs, dir);
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteFailsIfNonRecursive() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
-
-        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        final Path someDir2 = new Path(fsHome, "/someDir1/someDir2");
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.delete(someDir2, false);
-
-                return null;
-            }
-        }, PathIsNotEmptyDirectoryException.class, null);
-
-        assertPathExists(fs, someDir2);
-        assertPathExists(fs, someDir3);
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteRecursively() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
-
-        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        Path someDir2 = new Path(fsHome, "/someDir1/someDir2");
-
-        assertTrue(fs.delete(someDir2, true));
-
-        assertPathDoesNotExist(fs, someDir2);
-        assertPathDoesNotExist(fs, someDir3);
-    }
-
-    /** @throws Exception If failed. */
-    public void testDeleteRecursivelyFromRoot() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path someDir3 = new Path(fsHome, "/someDir1/someDir2/someDir3");
-
-        FSDataOutputStream os = fs.create(someDir3, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        Path root = new Path(fsHome, "/");
-
-        assertTrue(fs.delete(root, true));
-
-        assertPathDoesNotExist(fs, someDir3);
-        assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1/someDir2"));
-        assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1"));
-        assertPathExists(fs, root);
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetPermissionCheckDefaultPermission() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        fs.setPermission(file, null);
-
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file).getPermission());
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file.getParent()).getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetPermissionCheckNonRecursiveness() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        Path tmpDir = new Path(fsHome, "/tmp");
-
-        FsPermission perm = new FsPermission((short)123);
-
-        fs.setPermission(tmpDir, perm);
-
-        assertEquals(perm, fs.getFileStatus(tmpDir).getPermission());
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(file).getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    @SuppressWarnings("OctalInteger")
-    public void testSetPermission() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        for (short i = 0; i <= 0777; i += 7) {
-            FsPermission perm = new FsPermission(i);
-
-            fs.setPermission(file, perm);
-
-            assertEquals(perm, fs.getFileStatus(file).getPermission());
-        }
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetPermissionIfOutputStreamIsNotClosed() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "myFile");
-
-        FsPermission perm = new FsPermission((short)123);
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        fs.setPermission(file, perm);
-
-        os.close();
-
-        assertEquals(perm, fs.getFileStatus(file).getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwnerCheckParametersPathIsNull() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.setOwner(null, "aUser", "aGroup");
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: p");
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwnerCheckParametersUserIsNull() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.setOwner(file, null, "aGroup");
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: username");
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwnerCheckParametersGroupIsNull() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
-                fs.setOwner(file, "aUser", null);
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: grpName");
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwner() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        fs.setOwner(file, "aUser", "aGroup");
-
-        assertEquals("aUser", fs.getFileStatus(file).getOwner());
-        assertEquals("aGroup", fs.getFileStatus(file).getGroup());
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwnerIfOutputStreamIsNotClosed() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "myFile");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        fs.setOwner(file, "aUser", "aGroup");
-
-        os.close();
-
-        assertEquals("aUser", fs.getFileStatus(file).getOwner());
-        assertEquals("aGroup", fs.getFileStatus(file).getGroup());
-    }
-
-    /** @throws Exception If failed. */
-    public void testSetOwnerCheckNonRecursiveness() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "/tmp/my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        Path tmpDir = new Path(fsHome, "/tmp");
-
-        fs.setOwner(file, "fUser", "fGroup");
-        fs.setOwner(tmpDir, "dUser", "dGroup");
-
-        assertEquals("dUser", fs.getFileStatus(tmpDir).getOwner());
-        assertEquals("dGroup", fs.getFileStatus(tmpDir).getGroup());
-
-        assertEquals("fUser", fs.getFileStatus(file).getOwner());
-        assertEquals("fGroup", fs.getFileStatus(file).getGroup());
-    }
-
-    /** @throws Exception If failed. */
-    public void testOpenCheckParametersPathIsNull() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.open(null, 1024);
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testOpenNoSuchPath() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "someFile");
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.open(file, 1024);
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testOpenIfPathIsAlreadyOpened() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "someFile");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        FSDataInputStream is1 = fs.open(file);
-        FSDataInputStream is2 = fs.open(file);
-
-        is1.close();
-        is2.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testOpen() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "someFile");
-
-        int cnt = 2 * 1024;
-
-        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        for (long i = 0; i < cnt; i++)
-            out.writeLong(i);
-
-        out.close();
-
-        FSDataInputStream in = fs.open(file, 1024);
-
-        for (long i = 0; i < cnt; i++)
-            assertEquals(i, in.readLong());
-
-        in.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testAppendIfPathPointsToDirectory() throws Exception {
-        final Path fsHome = new Path(primaryFsUri);
-        final Path dir = new Path(fsHome, "/tmp");
-        Path file = new Path(dir, "my");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.create(new Path(fsHome, dir), EnumSet.of(CreateFlag.APPEND),
-                    Options.CreateOpts.perms(FsPermission.getDefault()));
-            }
-        }, IOException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testAppendIfFileIsAlreadyBeingOpenedToWrite() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "someFile");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        FSDataOutputStream appendOs = fs.create(file, EnumSet.of(CreateFlag.APPEND),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
-                return fs.create(file, EnumSet.of(CreateFlag.APPEND),
-                    Options.CreateOpts.perms(FsPermission.getDefault()));
-            }
-        }, IOException.class, null);
-
-        appendOs.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testAppend() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path file = new Path(fsHome, "someFile");
-
-        int cnt = 1024;
-
-        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        for (int i = 0; i < cnt; i++)
-            out.writeLong(i);
-
-        out.close();
-
-        out = fs.create(file, EnumSet.of(CreateFlag.APPEND),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        for (int i = cnt; i < cnt * 2; i++)
-            out.writeLong(i);
-
-        out.close();
-
-        FSDataInputStream in = fs.open(file, 1024);
-
-        for (int i = 0; i < cnt * 2; i++)
-            assertEquals(i, in.readLong());
-
-        in.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameCheckParametersSrcPathIsNull() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "someFile");
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.rename(null, file);
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameCheckParametersDstPathIsNull() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path file = new Path(fsHome, "someFile");
-
-        fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault())).close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
-                fs.rename(file, null);
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameIfSrcPathDoesNotExist() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path srcFile = new Path(fsHome, "srcFile");
-        final Path dstFile = new Path(fsHome, "dstFile");
-
-        assertPathDoesNotExist(fs, srcFile);
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.rename(srcFile, dstFile);
-
-                return null;
-            }
-        }, FileNotFoundException.class, null);
-
-        assertPathDoesNotExist(fs, dstFile);
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameIfSrcPathIsAlreadyBeingOpenedToWrite() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path srcFile = new Path(fsHome, "srcFile");
-        Path dstFile = new Path(fsHome, "dstFile");
-
-        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        os = fs.create(srcFile, EnumSet.of(CreateFlag.APPEND),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        fs.rename(srcFile, dstFile);
-
-        assertPathExists(fs, dstFile);
-
-        String testStr = "Test";
-
-        try {
-            os.writeBytes(testStr);
-        }
-        finally {
-            os.close();
-        }
-
-        try (FSDataInputStream is = fs.open(dstFile)) {
-            byte[] buf = new byte[testStr.getBytes().length];
-
-            is.readFully(buf);
-
-            assertEquals(testStr, new String(buf));
-        }
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameFileIfDstPathExists() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        final Path srcFile = new Path(fsHome, "srcFile");
-        final Path dstFile = new Path(fsHome, "dstFile");
-
-        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        os = fs.create(dstFile, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.rename(srcFile, dstFile);
-
-                return null;
-            }
-        }, FileAlreadyExistsException.class, null);
-
-        assertPathExists(fs, srcFile);
-        assertPathExists(fs, dstFile);
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameFile() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path srcFile = new Path(fsHome, "/tmp/srcFile");
-        Path dstFile = new Path(fsHome, "/tmp/dstFile");
-
-        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        fs.rename(srcFile, dstFile);
-
-        assertPathDoesNotExist(fs, srcFile);
-        assertPathExists(fs, dstFile);
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameIfSrcPathIsAlreadyBeingOpenedToRead() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path srcFile = new Path(fsHome, "srcFile");
-        Path dstFile = new Path(fsHome, "dstFile");
-
-        FSDataOutputStream os = fs.create(srcFile, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        int cnt = 1024;
-
-        for (int i = 0; i < cnt; i++)
-            os.writeInt(i);
-
-        os.close();
-
-        FSDataInputStream is = fs.open(srcFile);
-
-        for (int i = 0; i < cnt; i++) {
-            if (i == 100)
-                // Rename file during the read process.
-                fs.rename(srcFile, dstFile);
-
-            assertEquals(i, is.readInt());
-        }
-
-        assertPathDoesNotExist(fs, srcFile);
-        assertPathExists(fs, dstFile);
-
-        os.close();
-        is.close();
-    }
-
-    /** @throws Exception If failed. */
-    public void _testRenameDirectoryIfDstPathExists() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path srcDir = new Path(fsHome, "/tmp/");
-        Path dstDir = new Path(fsHome, "/tmpNew/");
-
-        FSDataOutputStream os = fs.create(new Path(srcDir, "file1"), EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        os = fs.create(new Path(dstDir, "file2"), EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        fs.rename(srcDir, dstDir);
-
-        assertPathExists(fs, dstDir);
-        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp"));
-        assertPathExists(fs, new Path(fsHome, "/tmpNew/tmp/file1"));
-    }
-
-    /** @throws Exception If failed. */
-    public void testRenameDirectory() throws Exception {
-        Path fsHome = new Path(primaryFsUri);
-        Path dir = new Path(fsHome, "/tmp/");
-        Path newDir = new Path(fsHome, "/tmpNew/");
-
-        FSDataOutputStream os = fs.create(new Path(dir, "myFile"), EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        os.close();
-
-        fs.rename(dir, newDir);
-
-        assertPathDoesNotExist(fs, dir);
-        assertPathExists(fs, newDir);
-    }
-
-    /** @throws Exception If failed. */
-    public void testListStatusIfPathIsNull() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.listStatus(null);
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testListStatusIfPathDoesNotExist() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.listStatus(new Path("/someDir"));
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /**
-     * Test directory listing.
-     *
-     * @throws Exception If failed.
-     */
-    public void testListStatus() throws Exception {
-        Path igfsHome = new Path(primaryFsUri);
-
-        // Test listing of an empty directory.
-        Path dir = new Path(igfsHome, "dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        FileStatus[] list = fs.listStatus(dir);
-
-        assert list.length == 0;
-
-        // Test listing of a not empty directory.
-        Path subDir = new Path(dir, "subDir");
-
-        fs.mkdir(subDir, FsPermission.getDefault(), true);
-
-        Path file = new Path(dir, "file");
-
-        FSDataOutputStream fos = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        fos.close();
-
-        list = fs.listStatus(dir);
-
-        assert list.length == 2;
-
-        String listRes1 = list[0].getPath().getName();
-        String listRes2 = list[1].getPath().getName();
-
-        assert "subDir".equals(listRes1) && "file".equals(listRes2) || "subDir".equals(listRes2) &&
-            "file".equals(listRes1);
-
-        // Test listing of a file.
-        list = fs.listStatus(file);
-
-        assert list.length == 1;
-
-        assert "file".equals(list[0].getPath().getName());
-    }
-
-    /** @throws Exception If failed. */
-    public void testMkdirsIfPathIsNull() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.mkdir(null, FsPermission.getDefault(), true);
-
-                return null;
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testMkdirsIfPermissionIsNull() throws Exception {
-        Path dir = new Path("/tmp");
-
-        fs.mkdir(dir, null, true);
-
-        assertEquals(FsPermission.getDefault(), fs.getFileStatus(dir).getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    @SuppressWarnings("OctalInteger")
-    public void testMkdirs() throws Exception {
-        Path fsHome = new Path(primaryFileSystemUriPath());
-        Path dir = new Path(fsHome, "/tmp/staging");
-        Path nestedDir = new Path(dir, "nested");
-
-        FsPermission dirPerm = FsPermission.createImmutable((short)0700);
-        FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
-
-        fs.mkdir(dir, dirPerm, true);
-        fs.mkdir(nestedDir, nestedDirPerm, true);
-
-        assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
-        assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
-    }
-
-    /** @throws Exception If failed. */
-    public void testGetFileStatusIfPathIsNull() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.getFileStatus(null);
-            }
-        }, NullPointerException.class, "Ouch! Argument cannot be null: f");
-    }
-
-    /** @throws Exception If failed. */
-    public void testGetFileStatusIfPathDoesNotExist() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.getFileStatus(new Path("someDir"));
-            }
-        }, FileNotFoundException.class, "File not found: someDir");
-    }
-
-    /** @throws Exception If failed. */
-    public void testGetFileBlockLocationsIfFileStatusIsNull() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                // Argument is checked by Hadoop.
-                return fs.getFileBlockLocations(null, 1, 2);
-            }
-        }, NullPointerException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testGetFileBlockLocationsIfFileStatusReferenceNotExistingPath() throws Exception {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.getFileBlockLocations(new Path("/someFile"), 1, 2);
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testGetFileBlockLocations() throws Exception {
-        Path igfsHome = new Path(primaryFsUri);
-
-        Path file = new Path(igfsHome, "someFile");
-
-        try (OutputStream out = new BufferedOutputStream(fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault())))) {
-            byte[] data = new byte[128 * 1024];
-
-            for (int i = 0; i < 100; i++)
-                out.write(data);
-
-            out.flush();
-        }
-
-        try (FSDataInputStream in = fs.open(file, 1024 * 1024)) {
-            byte[] data = new byte[128 * 1024];
-
-            int read;
-
-            do {
-                read = in.read(data);
-            }
-            while (read > 0);
-        }
-
-        FileStatus status = fs.getFileStatus(file);
-
-        int grpLen = 128 * 512 * 1024;
-
-        int grpCnt = (int)((status.getLen() + grpLen - 1) / grpLen);
-
-        BlockLocation[] locations = fs.getFileBlockLocations(file, 0, status.getLen());
-
-        assertEquals(grpCnt, locations.length);
-    }
-
-    /** @throws Exception If failed. */
-    public void testZeroReplicationFactor() throws Exception {
-        // This test doesn't make sense for any mode except of PRIMARY.
-        if (mode == PRIMARY) {
-            Path igfsHome = new Path(primaryFsUri);
-
-            Path file = new Path(igfsHome, "someFile");
-
-            try (FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-                Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.repFac((short)1))) {
-                out.write(new byte[1024 * 1024]);
-            }
-
-            IgniteFs igfs = grid(0).fileSystem("igfs");
-
-            IgfsPath filePath = new IgfsPath("/someFile");
-
-            IgfsFile fileInfo = igfs.info(filePath);
-
-            Collection<IgfsBlockLocation> locations = igfs.affinity(filePath, 0, fileInfo.length());
-
-            assertEquals(1, locations.size());
-
-            IgfsBlockLocation location = F.first(locations);
-
-            assertEquals(1, location.nodeIds().size());
-        }
-    }
-
-    /**
-     * Ensure that when running in multithreaded mode only one create() operation succeed.
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedCreate() throws Exception {
-        Path dir = new Path(new Path(primaryFsUri), "/dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        final Path file = new Path(dir, "file");
-
-        fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault())).close();
-
-        final AtomicInteger cnt = new AtomicInteger();
-
-        final Collection<Integer> errs = new GridConcurrentHashSet<>(THREAD_CNT, 1.0f, THREAD_CNT);
-
-        multithreaded(new Runnable() {
-            @Override public void run() {
-                int idx = cnt.getAndIncrement();
-
-                byte[] data = new byte[256];
-
-                Arrays.fill(data, (byte)idx);
-
-                FSDataOutputStream os = null;
-
-                try {
-                    os = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
-                        Options.CreateOpts.perms(FsPermission.getDefault()));
-
-                    os.write(data);
-                }
-                catch (IOException ignore) {
-                    errs.add(idx);
-                }
-                finally {
-                    U.awaitQuiet(barrier);
-
-                    U.closeQuiet(os);
-                }
-            }
-        }, THREAD_CNT);
-
-        // Only one thread could obtain write lock on the file.
-        assert errs.size() == THREAD_CNT - 1 : "Invalid errors count [expected=" + (THREAD_CNT - 1) + ", actual=" +
-            errs.size() + ']';
-
-        int idx = -1;
-
-        for (int i = 0; i < THREAD_CNT; i++) {
-            if (!errs.remove(i)) {
-                idx = i;
-
-                break;
-            }
-        }
-
-        byte[] expData = new byte[256];
-
-        Arrays.fill(expData, (byte)idx);
-
-        FSDataInputStream is = fs.open(file);
-
-        byte[] data = new byte[256];
-
-        is.read(data);
-
-        is.close();
-
-        assert Arrays.equals(expData, data);
-    }
-
-    /**
-     * Ensure that when running in multithreaded mode only one append() operation succeed.
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedAppend() throws Exception {
-        Path dir = new Path(new Path(primaryFsUri), "/dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        final Path file = new Path(dir, "file");
-
-        fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault())).close();
-
-        final AtomicInteger cnt = new AtomicInteger();
-
-        final Collection<Integer> errs = new GridConcurrentHashSet<>(THREAD_CNT, 1.0f, THREAD_CNT);
-
-        multithreaded(new Runnable() {
-            @Override public void run() {
-                int idx = cnt.getAndIncrement();
-
-                byte[] data = new byte[256];
-
-                Arrays.fill(data, (byte)idx);
-
-                U.awaitQuiet(barrier);
-
-                FSDataOutputStream os = null;
-
-                try {
-                    os = fs.create(file, EnumSet.of(CreateFlag.APPEND),
-                        Options.CreateOpts.perms(FsPermission.getDefault()));
-
-                    os.write(data);
-                }
-                catch (IOException ignore) {
-                    errs.add(idx);
-                }
-                finally {
-                    U.awaitQuiet(barrier);
-
-                    U.closeQuiet(os);
-                }
-            }
-        }, THREAD_CNT);
-
-        // Only one thread could obtain write lock on the file.
-        assert errs.size() == THREAD_CNT - 1;
-
-        int idx = -1;
-
-        for (int i = 0; i < THREAD_CNT; i++) {
-            if (!errs.remove(i)) {
-                idx = i;
-
-                break;
-            }
-        }
-
-        byte[] expData = new byte[256];
-
-        Arrays.fill(expData, (byte)idx);
-
-        FSDataInputStream is = fs.open(file);
-
-        byte[] data = new byte[256];
-
-        is.read(data);
-
-        is.close();
-
-        assert Arrays.equals(expData, data);
-    }
-
-    /**
-     * Test concurrent reads within the file.
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedOpen() throws Exception {
-        final byte[] dataChunk = new byte[256];
-
-        for (int i = 0; i < dataChunk.length; i++)
-            dataChunk[i] = (byte)i;
-
-        Path dir = new Path(new Path(primaryFsUri), "/dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        final Path file = new Path(dir, "file");
-
-        FSDataOutputStream os = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
-
-        // Write 256 * 2048 = 512Kb of data.
-        for (int i = 0; i < 2048; i++)
-            os.write(dataChunk);
-
-        os.close();
-
-        final AtomicBoolean err = new AtomicBoolean();
-
-        multithreaded(new Runnable() {
-            @Override
-            public void run() {
-                FSDataInputStream is = null;
-
-                try {
-                    int pos = ThreadLocalRandom8.current().nextInt(2048);
-
-                    try {
-                        is = fs.open(file);
-                    }
-                    finally {
-                        U.awaitQuiet(barrier);
-                    }
-
-                    is.seek(256 * pos);
-
-                    byte[] buf = new byte[256];
-
-                    for (int i = pos; i < 2048; i++) {
-                        // First perform normal read.
-                        int read = is.read(buf);
-
-                        assert read == 256;
-
-                        Arrays.equals(dataChunk, buf);
-                    }
-
-                    int res = is.read(buf);
-
-                    assert res == -1;
-                }
-                catch (IOException ignore) {
-                    err.set(true);
-                }
-                finally {
-                    U.closeQuiet(is);
-                }
-            }
-        }, THREAD_CNT);
-
-        assert !err.get();
-    }
-
-    /**
-     * Test concurrent creation of multiple directories.
-     *
-     * @throws Exception If failed.
-     */
-    public void testMultithreadedMkdirs() throws Exception {
-        final Path dir = new Path(new Path("igfs:///"), "/dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        final int depth = 3;
-        final int entryCnt = 5;
-
-        final AtomicBoolean err = new AtomicBoolean();
-
-        multithreaded(new Runnable() {
-            @Override public void run() {
-                Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
-
-                queue.add(F.t(0, dir));
-
-                U.awaitQuiet(barrier);
-
-                while (!queue.isEmpty()) {
-                    IgniteBiTuple<Integer, Path> t = queue.pollFirst();
-
-                    int curDepth = t.getKey();
-                    Path curPath = t.getValue();
-
-                    if (curDepth <= depth) {
-                        int newDepth = curDepth + 1;
-
-                        // Create directories.
-                        for (int i = 0; i < entryCnt; i++) {
-                            Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
-
-                            try {
-                                fs.mkdir(subDir, FsPermission.getDefault(), true);
-                            }
-                            catch (IOException ignore) {
-                                err.set(true);
-                            }
-
-                            queue.addLast(F.t(newDepth, subDir));
-                        }
-                    }
-                }
-            }
-        }, THREAD_CNT);
-
-        // Ensure there were no errors.
-        assert !err.get();
-
-        // Ensure correct folders structure.
-        Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
-
-        queue.add(F.t(0, dir));
-
-        while (!queue.isEmpty()) {
-            IgniteBiTuple<Integer, Path> t = queue.pollFirst();
-
-            int curDepth = t.getKey();
-            Path curPath = t.getValue();
-
-            if (curDepth <= depth) {
-                int newDepth = curDepth + 1;
-
-                // Create directories.
-                for (int i = 0; i < entryCnt; i++) {
-                    Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
-
-                    assertNotNull(fs.getFileStatus(subDir));
-
-                    queue.add(F.t(newDepth, subDir));
-                }
-            }
-        }
-    }
-
-    /**
-     * Test concurrent deletion of the same directory with advanced structure.
-     *
-     * @throws Exception If failed.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testMultithreadedDelete() throws Exception {
-        final Path dir = new Path(new Path(primaryFsUri), "/dir");
-
-        fs.mkdir(dir, FsPermission.getDefault(), true);
-
-        int depth = 3;
-        int entryCnt = 5;
-
-        Deque<IgniteBiTuple<Integer, Path>> queue = new ArrayDeque<>();
-
-        queue.add(F.t(0, dir));
-
-        while (!queue.isEmpty()) {
-            IgniteBiTuple<Integer, Path> t = queue.pollFirst();
-
-            int curDepth = t.getKey();
-            Path curPath = t.getValue();
-
-            if (curDepth < depth) {
-                int newDepth = curDepth + 1;
-
-                // Create directories.
-                for (int i = 0; i < entryCnt; i++) {
-                    Path subDir = new Path(curPath, "dir-" + newDepth + "-" + i);
-
-                    fs.mkdir(subDir, FsPermission.getDefault(), true);
-
-                    queue.addLast(F.t(newDepth, subDir));
-                }
-            }
-            else {
-                // Create files.
-                for (int i = 0; i < entryCnt; i++) {
-                    Path file = new Path(curPath, "file " + i);
-
-                    fs.create(file, EnumSet.noneOf(CreateFlag.class),
-                        Options.CreateOpts.perms(FsPermission.getDefault())).close();
-                }
-            }
-        }
-
-        final AtomicBoolean err = new AtomicBoolean();
-
-        multithreaded(new Runnable() {
-            @Override public void run() {
-                try {
-                    U.awaitQuiet(barrier);
-
-                    fs.delete(dir, true);
-                }
-                catch (FileNotFoundException ignore) {
-                    // No-op.
-                }
-                catch (IOException ignore) {
-                    err.set(true);
-                }
-            }
-        }, THREAD_CNT);
-
-        // Ensure there were no errors.
-        assert !err.get();
-
-        // Ensure the directory was actually deleted.
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                fs.getFileStatus(dir);
-
-                return null;
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /** @throws Exception If failed. */
-    public void testConsistency() throws Exception {
-        // Default buffers values
-        checkConsistency(-1, 1, -1, -1, 1, -1);
-        checkConsistency(-1, 10, -1, -1, 10, -1);
-        checkConsistency(-1, 100, -1, -1, 100, -1);
-        checkConsistency(-1, 1000, -1, -1, 1000, -1);
-        checkConsistency(-1, 10000, -1, -1, 10000, -1);
-        checkConsistency(-1, 100000, -1, -1, 100000, -1);
-
-        checkConsistency(65 * 1024 + 13, 100000, -1, -1, 100000, -1);
-
-        checkConsistency(-1, 100000, 2 * 4 * 1024 + 17, -1, 100000, -1);
-
-        checkConsistency(-1, 100000, -1, 65 * 1024 + 13, 100000, -1);
-
-        checkConsistency(-1, 100000, -1, -1, 100000, 2 * 4 * 1024 + 17);
-
-        checkConsistency(65 * 1024 + 13, 100000, 2 * 4 * 1024 + 13, 65 * 1024 + 149, 100000, 2 * 4 * 1024 + 157);
-    }
-
-    /**
-     * Verifies that client reconnects after connection to the server has been lost.
-     *
-     * @throws Exception If error occurs.
-     */
-    public void testClientReconnect() throws Exception {
-        final Path igfsHome = new Path(primaryFsUri);
-
-        final Path filePath = new Path(igfsHome, "someFile");
-
-        final FSDataOutputStream s = fs.create(filePath, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault())); // Open stream before stopping IGFS.
-
-        try {
-            G.stopAll(true); // Stop the server.
-
-            startNodes(); // Start server again.
-
-            // Check that client is again operational.
-            fs.mkdir(new Path("igfs:///dir1/dir2"), FsPermission.getDefault(), true);
-
-            // However, the streams, opened before disconnect, should not be valid.
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Nullable @Override public Object call() throws Exception {
-                    s.write("test".getBytes());
-
-                    s.flush();
-
-                    return null;
-                }
-            }, IOException.class, null);
-
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    fs.getFileStatus(filePath);
-
-                    return null;
-                }
-            }, FileNotFoundException.class, null);
-        }
-        finally {
-            U.closeQuiet(s);
-        }
-    }
-
-    /**
-     * Verifies that client reconnects after connection to the server has been lost (multithreaded mode).
-     *
-     * @throws Exception If error occurs.
-     */
-    public void testClientReconnectMultithreaded() throws Exception {
-        final ConcurrentLinkedQueue<FileSystem> q = new ConcurrentLinkedQueue<>();
-
-        Configuration cfg = new Configuration();
-
-        for (Map.Entry<String, String> entry : primaryFsCfg)
-            cfg.set(entry.getKey(), entry.getValue());
-
-        cfg.setBoolean("fs.igfs.impl.disable.cache", true);
-
-        final int nClients = 16;
-
-        // Initialize clients.
-        for (int i = 0; i < nClients; i++)
-            q.add(FileSystem.get(primaryFsUri, cfg));
-
-        G.stopAll(true); // Stop the server.
-
-        startNodes(); // Start server again.
-
-        GridTestUtils.runMultiThreaded(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                FileSystem fs = q.poll();
-
-                try {
-                    // Check that client is again operational.
-                    assertTrue(fs.mkdirs(new Path("igfs:///" + Thread.currentThread().getName())));
-
-                    return true;
-                }
-                finally {
-                    U.closeQuiet(fs);
-                }
-            }
-        }, nClients, "test-client");
-    }
-
-    /**
-     * Checks consistency of create --> open --> append --> open operations with different buffer sizes.
-     *
-     * @param createBufSize Buffer size used for file creation.
-     * @param writeCntsInCreate Count of times to write in file creation.
-     * @param openAfterCreateBufSize Buffer size used for file opening after creation.
-     * @param appendBufSize Buffer size used for file appending.
-     * @param writeCntsInAppend Count of times to write in file appending.
-     * @param openAfterAppendBufSize Buffer size used for file opening after appending.
-     * @throws Exception If failed.
-     */
-    private void checkConsistency(int createBufSize, int writeCntsInCreate, int openAfterCreateBufSize,
-        int appendBufSize, int writeCntsInAppend, int openAfterAppendBufSize) throws Exception {
-        final Path igfsHome = new Path(primaryFsUri);
-
-        Path file = new Path(igfsHome, "/someDir/someInnerDir/someFile");
-
-        if (createBufSize == -1)
-            createBufSize = fs.getServerDefaults().getFileBufferSize();
-
-        if (appendBufSize == -1)
-            appendBufSize = fs.getServerDefaults().getFileBufferSize();
-
-        FSDataOutputStream os = fs.create(file, EnumSet.of(CreateFlag.OVERWRITE),
-            Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.bufferSize(createBufSize));
-
-        for (int i = 0; i < writeCntsInCreate; i++)
-            os.writeInt(i);
-
-        os.close();
-
-        FSDataInputStream is = fs.open(file, openAfterCreateBufSize);
-
-        for (int i = 0; i < writeCntsInCreate; i++)
-            assertEquals(i, is.readInt());
-
-        is.close();
-
-        os = fs.create(file, EnumSet.of(CreateFlag.APPEND),
-            Options.CreateOpts.perms(FsPermission.getDefault()), Options.CreateOpts.bufferSize(appendBufSize));
-
-        for (int i = writeCntsInCreate; i < writeCntsInCreate + writeCntsInAppend; i++)
-            os.writeInt(i);
-
-        os.close();
-
-        is = fs.open(file, openAfterAppendBufSize);
-
-        for (int i = 0; i < writeCntsInCreate + writeCntsInAppend; i++)
-            assertEquals(i, is.readInt());
-
-        is.close();
-    }
-
-    /**
-     * Test expected failures for 'close' operation.
-     *
-     * @param fs File system to test.
-     * @param msg Expected exception message.
-     */
-    public void assertCloseFails(final FileSystem fs, String msg) {
-        GridTestUtils.assertThrows(log, new Callable() {
-            @Override public Object call() throws Exception {
-                fs.close();
-
-                return null;
-            }
-        }, IOException.class, msg);
-    }
-
-    /**
-     * Test expected failures for 'get content summary' operation.
-     *
-     * @param fs File system to test.
-     * @param path Path to evaluate content summary for.
-     */
-    private void assertContentSummaryFails(final FileSystem fs, final Path path) {
-        GridTestUtils.assertThrows(log, new Callable<ContentSummary>() {
-            @Override public ContentSummary call() throws Exception {
-                return fs.getContentSummary(path);
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /**
-     * Assert that a given path exists in a given FileSystem.
-     *
-     * @param fs FileSystem to check.
-     * @param p Path to check.
-     * @throws IOException if the path does not exist.
-     */
-    private void assertPathExists(AbstractFileSystem fs, Path p) throws IOException {
-        FileStatus fileStatus = fs.getFileStatus(p);
-
-        assertEquals(p, fileStatus.getPath());
-        assertNotSame(0, fileStatus.getModificationTime());
-    }
-
-    /**
-     * Check path does not exist in a given FileSystem.
-     *
-     * @param fs FileSystem to check.
-     * @param path Path to check.
-     */
-    private void assertPathDoesNotExist(final AbstractFileSystem fs, final Path path) {
-        GridTestUtils.assertThrows(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                return fs.getFileStatus(path);
-            }
-        }, FileNotFoundException.class, null);
-    }
-
-    /** Helper class to encapsulate source and destination folders. */
-    @SuppressWarnings({"PublicInnerClass", "PublicField"})
-    public static final class Config {
-        /** Source file system. */
-        public final AbstractFileSystem srcFs;
-
-        /** Source path to work with. */
-        public final Path src;
-
-        /** Destination file system. */
-        public final AbstractFileSystem destFs;
-
-        /** Destination path to work with. */
-        public final Path dest;
-
-        /**
-         * Copying task configuration.
-         *
-         * @param srcFs Source file system.
-         * @param src Source path.
-         * @param destFs Destination file system.
-         * @param dest Destination path.
-         */
-        public Config(AbstractFileSystem srcFs, Path src, AbstractFileSystem destFs, Path dest) {
-            this.srcFs = srcFs;
-            this.src = src;
-            this.destFs = destFs;
-            this.dest = dest;
-        }
-    }
-
-    /**
-     * Convert path for exception message testing purposes.
-     *
-     * @param path Path.
-     * @return Converted path.
-     * @throws Exception If failed.
-     */
-    private Path convertPath(Path path) throws Exception {
-        if (mode != PROXY)
-            return path;
-        else {
-            URI secondaryUri = new URI(secondaryFileSystemUriPath());
-
-            URI pathUri = path.toUri();
-
-            return new Path(new URI(pathUri.getScheme() != null ? secondaryUri.getScheme() : null,
-                pathUri.getAuthority() != null ? secondaryUri.getAuthority() : null, pathUri.getPath(), null, null));
-        }
-    }
-}


[09/31] incubator-ignite git commit: # IGNITE-386: WIP on internal namings (2).

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopPrepareForJobRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopPrepareForJobRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopPrepareForJobRequest.java
deleted file mode 100644
index 3a55d19..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopPrepareForJobRequest.java
+++ /dev/null
@@ -1,126 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Child process initialization request.
- */
-public class GridHadoopPrepareForJobRequest implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Job ID. */
-    @GridToStringInclude
-    private GridHadoopJobId jobId;
-
-    /** Job info. */
-    @GridToStringInclude
-    private GridHadoopJobInfo jobInfo;
-
-    /** Total amount of reducers in the job. */
-    @GridToStringInclude
-    private int totalReducersCnt;
-
-    /** Reducers to be executed on current node. */
-    @GridToStringInclude
-    private int[] locReducers;
-
-    /**
-     * Constructor required by {@link Externalizable}.
-     */
-    public GridHadoopPrepareForJobRequest() {
-        // No-op.
-    }
-
-    /**
-     * @param jobId Job ID.
-     * @param jobInfo Job info.
-     * @param totalReducersCnt Number of reducers in the job.
-     * @param locReducers Reducers to be executed on current node.
-     */
-    public GridHadoopPrepareForJobRequest(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo, int totalReducersCnt,
-        int[] locReducers) {
-        assert jobId != null;
-
-        this.jobId = jobId;
-        this.jobInfo = jobInfo;
-        this.totalReducersCnt = totalReducersCnt;
-        this.locReducers = locReducers;
-    }
-
-    /**
-     * @return Job info.
-     */
-    public GridHadoopJobInfo jobInfo() {
-        return jobInfo;
-    }
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @return Reducers to be executed on current node.
-     */
-    public int[] localReducers() {
-        return locReducers;
-    }
-
-    /**
-     * @return Number of reducers in job.
-     */
-    public int totalReducerCount() {
-        return totalReducersCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        jobId.writeExternal(out);
-
-        out.writeObject(jobInfo);
-        out.writeInt(totalReducersCnt);
-
-        U.writeIntArray(out, locReducers);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = new GridHadoopJobId();
-        jobId.readExternal(in);
-
-        jobInfo = (GridHadoopJobInfo)in.readObject();
-        totalReducersCnt = in.readInt();
-
-        locReducers = U.readIntArray(in);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopPrepareForJobRequest.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessDescriptor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessDescriptor.java
deleted file mode 100644
index 7fc8858..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessDescriptor.java
+++ /dev/null
@@ -1,150 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Process descriptor used to identify process for which task is running.
- */
-public class GridHadoopProcessDescriptor implements Serializable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Parent node ID. */
-    private UUID parentNodeId;
-
-    /** Process ID. */
-    private UUID procId;
-
-    /** Address. */
-    private String addr;
-
-    /** TCP port. */
-    private int tcpPort;
-
-    /** Shared memory port. */
-    private int shmemPort;
-
-    /**
-     * @param parentNodeId Parent node ID.
-     * @param procId Process ID.
-     */
-    public GridHadoopProcessDescriptor(UUID parentNodeId, UUID procId) {
-        this.parentNodeId = parentNodeId;
-        this.procId = procId;
-    }
-
-    /**
-     * Gets process ID.
-     *
-     * @return Process ID.
-     */
-    public UUID processId() {
-        return procId;
-    }
-
-    /**
-     * Gets parent node ID.
-     *
-     * @return Parent node ID.
-     */
-    public UUID parentNodeId() {
-        return parentNodeId;
-    }
-
-    /**
-     * Gets host address.
-     *
-     * @return Host address.
-     */
-    public String address() {
-        return addr;
-    }
-
-    /**
-     * Sets host address.
-     *
-     * @param addr Host address.
-     */
-    public void address(String addr) {
-        this.addr = addr;
-    }
-
-    /**
-     * @return Shared memory port.
-     */
-    public int sharedMemoryPort() {
-        return shmemPort;
-    }
-
-    /**
-     * Sets shared memory port.
-     *
-     * @param shmemPort Shared memory port.
-     */
-    public void sharedMemoryPort(int shmemPort) {
-        this.shmemPort = shmemPort;
-    }
-
-    /**
-     * @return TCP port.
-     */
-    public int tcpPort() {
-        return tcpPort;
-    }
-
-    /**
-     * Sets TCP port.
-     *
-     * @param tcpPort TCP port.
-     */
-    public void tcpPort(int tcpPort) {
-        this.tcpPort = tcpPort;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-
-        if (!(o instanceof GridHadoopProcessDescriptor))
-            return false;
-
-        GridHadoopProcessDescriptor that = (GridHadoopProcessDescriptor)o;
-
-        return parentNodeId.equals(that.parentNodeId) && procId.equals(that.procId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int result = parentNodeId.hashCode();
-
-        result = 31 * result + procId.hashCode();
-
-        return result;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopProcessDescriptor.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessStartedAck.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessStartedAck.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessStartedAck.java
deleted file mode 100644
index 679da6c..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopProcessStartedAck.java
+++ /dev/null
@@ -1,46 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Process started message.
- */
-public class GridHadoopProcessStartedAck implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopProcessStartedAck.class, this);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskExecutionRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskExecutionRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskExecutionRequest.java
deleted file mode 100644
index 9f11e0e..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskExecutionRequest.java
+++ /dev/null
@@ -1,110 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Message sent from node to child process to start task(s) execution.
- */
-public class GridHadoopTaskExecutionRequest implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Job ID. */
-    @GridToStringInclude
-    private GridHadoopJobId jobId;
-
-    /** Job info. */
-    @GridToStringInclude
-    private GridHadoopJobInfo jobInfo;
-
-    /** Mappers. */
-    @GridToStringInclude
-    private Collection<GridHadoopTaskInfo> tasks;
-
-    /**
-     * @return Job ID.
-     */
-    public GridHadoopJobId jobId() {
-        return jobId;
-    }
-
-    /**
-     * @param jobId Job ID.
-     */
-    public void jobId(GridHadoopJobId jobId) {
-        this.jobId = jobId;
-    }
-
-    /**
-     * @return Jon info.
-     */
-    public GridHadoopJobInfo jobInfo() {
-        return jobInfo;
-    }
-
-    /**
-     * @param jobInfo Job info.
-     */
-    public void jobInfo(GridHadoopJobInfo jobInfo) {
-        this.jobInfo = jobInfo;
-    }
-
-    /**
-     * @return Tasks.
-     */
-    public Collection<GridHadoopTaskInfo> tasks() {
-        return tasks;
-    }
-
-    /**
-     * @param tasks Tasks.
-     */
-    public void tasks(Collection<GridHadoopTaskInfo> tasks) {
-        this.tasks = tasks;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopTaskExecutionRequest.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        jobId.writeExternal(out);
-
-        out.writeObject(jobInfo);
-        U.writeCollection(out, tasks);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        jobId = new GridHadoopJobId();
-        jobId.readExternal(in);
-
-        jobInfo = (GridHadoopJobInfo)in.readObject();
-        tasks = U.readCollection(in);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskFinishedMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskFinishedMessage.java
deleted file mode 100644
index f69abaf..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopTaskFinishedMessage.java
+++ /dev/null
@@ -1,92 +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.taskexecutor.external;
-
-import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.message.*;
-import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-
-import java.io.*;
-
-/**
- * Task finished message. Sent when local task finishes execution.
- */
-public class GridHadoopTaskFinishedMessage implements GridHadoopMessage {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Finished task info. */
-    private GridHadoopTaskInfo taskInfo;
-
-    /** Task finish status. */
-    private GridHadoopTaskStatus status;
-
-    /**
-     * Constructor required by {@link Externalizable}.
-     */
-    public GridHadoopTaskFinishedMessage() {
-        // No-op.
-    }
-
-    /**
-     * @param taskInfo Finished task info.
-     * @param status Task finish status.
-     */
-    public GridHadoopTaskFinishedMessage(GridHadoopTaskInfo taskInfo, GridHadoopTaskStatus status) {
-        assert taskInfo != null;
-        assert status != null;
-
-        this.taskInfo = taskInfo;
-        this.status = status;
-    }
-
-    /**
-     * @return Finished task info.
-     */
-    public GridHadoopTaskInfo taskInfo() {
-        return taskInfo;
-    }
-
-    /**
-     * @return Task finish status.
-     */
-    public GridHadoopTaskStatus status() {
-        return status;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridHadoopTaskFinishedMessage.class, this);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        taskInfo.writeExternal(out);
-        status.writeExternal(out);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        taskInfo = new GridHadoopTaskInfo();
-        taskInfo.readExternal(in);
-
-        status = new GridHadoopTaskStatus();
-        status.readExternal(in);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
index 616d383..10ad648 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
@@ -39,7 +39,7 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.locks.*;
 
-import static org.apache.ignite.internal.processors.hadoop.taskexecutor.GridHadoopTaskState.*;
+import static org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState.*;
 
 /**
  * External process registry. Handles external process lifecycle.
@@ -55,7 +55,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     private IgniteLogger log;
 
     /** Node process descriptor. */
-    private GridHadoopProcessDescriptor nodeDesc;
+    private HadoopProcessDescriptor nodeDesc;
 
     /** Output base. */
     private File outputBase;
@@ -127,7 +127,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onJobStateChanged(final GridHadoopJobMetadata meta) {
+    @Override public void onJobStateChanged(final HadoopJobMetadata meta) {
         final HadoopProcess proc = runningProcsByJobId.get(meta.jobId());
 
         // If we have a local process for this job.
@@ -156,8 +156,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                         "[jobId=" + meta.jobId() + ", meta=" + meta + ']');
             }
             else {
-                proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-                    @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>>>() {
+                    @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> f) {
                         try {
                             f.get();
 
@@ -223,9 +223,9 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
 
             final HadoopProcess proc0 = proc;
 
-            proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+            proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>>>() {
                 @Override public void apply(
-                    IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                    IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> f) {
                     if (!busyLock.tryReadLock())
                         return;
 
@@ -281,7 +281,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                 return;
             }
 
-            GridHadoopTaskExecutionRequest req = new GridHadoopTaskExecutionRequest();
+            HadoopTaskExecutionRequest req = new HadoopTaskExecutionRequest();
 
             req.jobId(job.id());
             req.jobInfo(job.info());
@@ -297,8 +297,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     /**
      * @return External task metadata.
      */
-    private GridHadoopExternalTaskMetadata buildTaskMeta() {
-        GridHadoopExternalTaskMetadata meta = new GridHadoopExternalTaskMetadata();
+    private HadoopExternalTaskMetadata buildTaskMeta() {
+        HadoopExternalTaskMetadata meta = new HadoopExternalTaskMetadata();
 
         meta.classpath(Arrays.asList(System.getProperty("java.class.path").split(File.pathSeparator)));
         meta.jvmOptions(Arrays.asList("-Xmx1g", "-ea", "-XX:+UseConcMarkSweepGC", "-XX:+CMSClassUnloadingEnabled",
@@ -312,8 +312,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param state Fail state.
      * @param e Optional error.
      */
-    private void notifyTasksFailed(Iterable<GridHadoopTaskInfo> tasks, GridHadoopTaskState state, Throwable e) {
-        GridHadoopTaskStatus fail = new GridHadoopTaskStatus(state, e);
+    private void notifyTasksFailed(Iterable<GridHadoopTaskInfo> tasks, HadoopTaskState state, Throwable e) {
+        HadoopTaskStatus fail = new HadoopTaskStatus(state, e);
 
         for (GridHadoopTaskInfo task : tasks)
             jobTracker.onTaskFinished(task, fail);
@@ -351,7 +351,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                 }
 
                 try {
-                    GridHadoopExternalTaskMetadata startMeta = buildTaskMeta();
+                    HadoopExternalTaskMetadata startMeta = buildTaskMeta();
 
                     if (log.isDebugEnabled())
                         log.debug("Created hadoop child process metadata for job [job=" + job +
@@ -404,8 +404,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
             }
         }, true);
 
-        fut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-            @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>>>() {
+            @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> f) {
                 try {
                     // Make sure there were no exceptions.
                     f.get();
@@ -493,7 +493,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param job Job.
      * @return Started process.
      */
-    private Process startJavaProcess(UUID childProcId, GridHadoopExternalTaskMetadata startMeta,
+    private Process startJavaProcess(UUID childProcId, HadoopExternalTaskMetadata startMeta,
         GridHadoopJob job) throws Exception {
         String outFldr = jobWorkFolder(job.id()) + File.separator + childProcId;
 
@@ -565,18 +565,18 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param proc Process to send request to.
      * @param meta Job metadata.
      */
-    private void sendJobInfoUpdate(HadoopProcess proc, GridHadoopJobMetadata meta) {
-        Map<Integer, GridHadoopProcessDescriptor> rdcAddrs = meta.reducersAddresses();
+    private void sendJobInfoUpdate(HadoopProcess proc, HadoopJobMetadata meta) {
+        Map<Integer, HadoopProcessDescriptor> rdcAddrs = meta.reducersAddresses();
 
         int rdcNum = meta.mapReducePlan().reducers();
 
-        GridHadoopProcessDescriptor[] addrs = null;
+        HadoopProcessDescriptor[] addrs = null;
 
         if (rdcAddrs != null && rdcAddrs.size() == rdcNum) {
-            addrs = new GridHadoopProcessDescriptor[rdcNum];
+            addrs = new HadoopProcessDescriptor[rdcNum];
 
             for (int i = 0; i < rdcNum; i++) {
-                GridHadoopProcessDescriptor desc = rdcAddrs.get(i);
+                HadoopProcessDescriptor desc = rdcAddrs.get(i);
 
                 assert desc != null : "Missing reducing address [meta=" + meta + ", rdc=" + i + ']';
 
@@ -585,7 +585,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         }
 
         try {
-            comm.sendMessage(proc.descriptor(), new GridHadoopJobInfoUpdateRequest(proc.jobId, meta.phase(), addrs));
+            comm.sendMessage(proc.descriptor(), new HadoopJobInfoUpdateRequest(proc.jobId, meta.phase(), addrs));
         }
         catch (IgniteCheckedException e) {
             if (!proc.terminated()) {
@@ -606,7 +606,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      */
     private void prepareForJob(HadoopProcess proc, GridHadoopJob job, GridHadoopMapReducePlan plan) {
         try {
-            comm.sendMessage(proc.descriptor(), new GridHadoopPrepareForJobRequest(job.id(), job.info(),
+            comm.sendMessage(proc.descriptor(), new HadoopPrepareForJobRequest(job.id(), job.info(),
                 plan.reducers(), plan.reducers(ctx.localNodeId())));
         }
         catch (IgniteCheckedException e) {
@@ -623,7 +623,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param desc Remote process descriptor.
      * @param taskMsg Task finished message.
      */
-    private void processTaskFinishedMessage(GridHadoopProcessDescriptor desc, GridHadoopTaskFinishedMessage taskMsg) {
+    private void processTaskFinishedMessage(HadoopProcessDescriptor desc, HadoopTaskFinishedMessage taskMsg) {
         HadoopProcess proc = runningProcsByProcId.get(desc.processId());
 
         if (proc != null)
@@ -637,12 +637,12 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      */
     private class MessageListener implements GridHadoopMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessageReceived(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
+        @Override public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg) {
             if (!busyLock.tryReadLock())
                 return;
 
             try {
-                if (msg instanceof GridHadoopProcessStartedAck) {
+                if (msg instanceof HadoopProcessStartedAck) {
                     HadoopProcess proc = runningProcsByProcId.get(desc.processId());
 
                     assert proc != null : "Missing child process for processId: " + desc;
@@ -655,8 +655,8 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                     else
                         log.warning("Failed to find process start future (will ignore): " + desc);
                 }
-                else if (msg instanceof GridHadoopTaskFinishedMessage) {
-                    GridHadoopTaskFinishedMessage taskMsg = (GridHadoopTaskFinishedMessage)msg;
+                else if (msg instanceof HadoopTaskFinishedMessage) {
+                    HadoopTaskFinishedMessage taskMsg = (HadoopTaskFinishedMessage)msg;
 
                     processTaskFinishedMessage(desc, taskMsg);
                 }
@@ -669,7 +669,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public void onConnectionLost(GridHadoopProcessDescriptor desc) {
+        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
             if (!busyLock.tryReadLock())
                 return;
 
@@ -689,7 +689,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                     if (!F.isEmpty(tasks)) {
                         log.warning("Lost connection with alive process (will terminate): " + desc);
 
-                        GridHadoopTaskStatus status = new GridHadoopTaskStatus(CRASHED,
+                        HadoopTaskStatus status = new HadoopTaskStatus(CRASHED,
                             new IgniteCheckedException("Failed to run tasks (external process finished unexpectedly): " + desc));
 
                         for (GridHadoopTaskInfo info : tasks)
@@ -725,7 +725,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         private final GridHadoopProcessFuture initFut;
 
         /** Process descriptor. */
-        private GridHadoopProcessDescriptor procDesc;
+        private HadoopProcessDescriptor procDesc;
 
         /** Reducers planned for this process. */
         private Collection<Integer> reducers;
@@ -756,7 +756,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         /**
          * @return Communication process descriptor.
          */
-        private GridHadoopProcessDescriptor descriptor() {
+        private HadoopProcessDescriptor descriptor() {
             return procDesc;
         }
 
@@ -773,7 +773,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
          * @param proc Java process representation.
          * @param procDesc Process descriptor.
          */
-        private void onInitialized(Process proc, GridHadoopProcessDescriptor procDesc) {
+        private void onInitialized(Process proc, HadoopProcessDescriptor procDesc) {
             this.proc = proc;
             this.procDesc = procDesc;
         }
@@ -789,9 +789,9 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
                 terminated = true;
 
                 if (!initFut.isDone())
-                    initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                    initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>>>() {
                         @Override public void apply(
-                            IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                            IgniteInternalFuture<IgniteBiTuple<Process, HadoopProcessDescriptor>> f) {
                             proc.destroy();
                         }
                     });
@@ -852,7 +852,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
     /**
      *
      */
-    private class GridHadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> {
+    private class GridHadoopProcessFuture extends GridFutureAdapter<IgniteBiTuple<Process, HadoopProcessDescriptor>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -863,7 +863,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         private GridHadoopJobId jobId;
 
         /** Process descriptor. */
-        private GridHadoopProcessDescriptor desc;
+        private HadoopProcessDescriptor desc;
 
         /** Running process. */
         private Process proc;
@@ -909,7 +909,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         /**
          * Reply received callback.
          */
-        public void onReplyReceived(GridHadoopProcessDescriptor desc) {
+        public void onReplyReceived(HadoopProcessDescriptor desc) {
             assert childProcId.equals(desc.processId());
 
             this.desc = desc;
@@ -921,7 +921,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public boolean onDone(@Nullable IgniteBiTuple<Process, GridHadoopProcessDescriptor> res,
+        @Override public boolean onDone(@Nullable IgniteBiTuple<Process, HadoopProcessDescriptor> res,
             @Nullable Throwable err) {
             if (err == null) {
                 HadoopProcess proc = runningProcsByProcId.get(childProcId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskMetadata.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskMetadata.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskMetadata.java
new file mode 100644
index 0000000..f0acc9f
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskMetadata.java
@@ -0,0 +1,68 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+
+/**
+ * External task metadata (classpath, JVM options) needed to start external process execution.
+ */
+public class HadoopExternalTaskMetadata {
+    /** Process classpath. */
+    private Collection<String> classpath;
+
+    /** JVM options. */
+    @GridToStringInclude
+    private Collection<String> jvmOpts;
+
+    /**
+     * @return JVM Options.
+     */
+    public Collection<String> jvmOptions() {
+        return jvmOpts;
+    }
+
+    /**
+     * @param jvmOpts JVM options.
+     */
+    public void jvmOptions(Collection<String> jvmOpts) {
+        this.jvmOpts = jvmOpts;
+    }
+
+    /**
+     * @return Classpath.
+     */
+    public Collection<String> classpath() {
+        return classpath;
+    }
+
+    /**
+     * @param classpath Classpath.
+     */
+    public void classpath(Collection<String> classpath) {
+        this.classpath = classpath;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopExternalTaskMetadata.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopJobInfoUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopJobInfoUpdateRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopJobInfoUpdateRequest.java
new file mode 100644
index 0000000..1258819
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopJobInfoUpdateRequest.java
@@ -0,0 +1,109 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Job info update request.
+ */
+public class HadoopJobInfoUpdateRequest implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job ID. */
+    @GridToStringInclude
+    private GridHadoopJobId jobId;
+
+    /** Job phase. */
+    @GridToStringInclude
+    private GridHadoopJobPhase jobPhase;
+
+    /** Reducers addresses. */
+    @GridToStringInclude
+    private HadoopProcessDescriptor[] reducersAddrs;
+
+    /**
+     * Constructor required by {@link Externalizable}.
+     */
+    public HadoopJobInfoUpdateRequest() {
+        // No-op.
+    }
+
+    /**
+     * @param jobId Job ID.
+     * @param jobPhase Job phase.
+     * @param reducersAddrs Reducers addresses.
+     */
+    public HadoopJobInfoUpdateRequest(GridHadoopJobId jobId, GridHadoopJobPhase jobPhase,
+        HadoopProcessDescriptor[] reducersAddrs) {
+        assert jobId != null;
+
+        this.jobId = jobId;
+        this.jobPhase = jobPhase;
+        this.reducersAddrs = reducersAddrs;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @return Job phase.
+     */
+    public GridHadoopJobPhase jobPhase() {
+        return jobPhase;
+    }
+
+    /**
+     * @return Reducers addresses.
+     */
+    public HadoopProcessDescriptor[] reducersAddresses() {
+        return reducersAddrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+
+        out.writeObject(jobPhase);
+        U.writeArray(out, reducersAddrs);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new GridHadoopJobId();
+        jobId.readExternal(in);
+
+        jobPhase = (GridHadoopJobPhase)in.readObject();
+        reducersAddrs = (HadoopProcessDescriptor[])U.readArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopJobInfoUpdateRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopPrepareForJobRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopPrepareForJobRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopPrepareForJobRequest.java
new file mode 100644
index 0000000..4037b26
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopPrepareForJobRequest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Child process initialization request.
+ */
+public class HadoopPrepareForJobRequest implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job ID. */
+    @GridToStringInclude
+    private GridHadoopJobId jobId;
+
+    /** Job info. */
+    @GridToStringInclude
+    private GridHadoopJobInfo jobInfo;
+
+    /** Total amount of reducers in the job. */
+    @GridToStringInclude
+    private int totalReducersCnt;
+
+    /** Reducers to be executed on current node. */
+    @GridToStringInclude
+    private int[] locReducers;
+
+    /**
+     * Constructor required by {@link Externalizable}.
+     */
+    public HadoopPrepareForJobRequest() {
+        // No-op.
+    }
+
+    /**
+     * @param jobId Job ID.
+     * @param jobInfo Job info.
+     * @param totalReducersCnt Number of reducers in the job.
+     * @param locReducers Reducers to be executed on current node.
+     */
+    public HadoopPrepareForJobRequest(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo, int totalReducersCnt,
+        int[] locReducers) {
+        assert jobId != null;
+
+        this.jobId = jobId;
+        this.jobInfo = jobInfo;
+        this.totalReducersCnt = totalReducersCnt;
+        this.locReducers = locReducers;
+    }
+
+    /**
+     * @return Job info.
+     */
+    public GridHadoopJobInfo jobInfo() {
+        return jobInfo;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @return Reducers to be executed on current node.
+     */
+    public int[] localReducers() {
+        return locReducers;
+    }
+
+    /**
+     * @return Number of reducers in job.
+     */
+    public int totalReducerCount() {
+        return totalReducersCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+
+        out.writeObject(jobInfo);
+        out.writeInt(totalReducersCnt);
+
+        U.writeIntArray(out, locReducers);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new GridHadoopJobId();
+        jobId.readExternal(in);
+
+        jobInfo = (GridHadoopJobInfo)in.readObject();
+        totalReducersCnt = in.readInt();
+
+        locReducers = U.readIntArray(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopPrepareForJobRequest.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessDescriptor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessDescriptor.java
new file mode 100644
index 0000000..dea73c3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessDescriptor.java
@@ -0,0 +1,150 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Process descriptor used to identify process for which task is running.
+ */
+public class HadoopProcessDescriptor implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Parent node ID. */
+    private UUID parentNodeId;
+
+    /** Process ID. */
+    private UUID procId;
+
+    /** Address. */
+    private String addr;
+
+    /** TCP port. */
+    private int tcpPort;
+
+    /** Shared memory port. */
+    private int shmemPort;
+
+    /**
+     * @param parentNodeId Parent node ID.
+     * @param procId Process ID.
+     */
+    public HadoopProcessDescriptor(UUID parentNodeId, UUID procId) {
+        this.parentNodeId = parentNodeId;
+        this.procId = procId;
+    }
+
+    /**
+     * Gets process ID.
+     *
+     * @return Process ID.
+     */
+    public UUID processId() {
+        return procId;
+    }
+
+    /**
+     * Gets parent node ID.
+     *
+     * @return Parent node ID.
+     */
+    public UUID parentNodeId() {
+        return parentNodeId;
+    }
+
+    /**
+     * Gets host address.
+     *
+     * @return Host address.
+     */
+    public String address() {
+        return addr;
+    }
+
+    /**
+     * Sets host address.
+     *
+     * @param addr Host address.
+     */
+    public void address(String addr) {
+        this.addr = addr;
+    }
+
+    /**
+     * @return Shared memory port.
+     */
+    public int sharedMemoryPort() {
+        return shmemPort;
+    }
+
+    /**
+     * Sets shared memory port.
+     *
+     * @param shmemPort Shared memory port.
+     */
+    public void sharedMemoryPort(int shmemPort) {
+        this.shmemPort = shmemPort;
+    }
+
+    /**
+     * @return TCP port.
+     */
+    public int tcpPort() {
+        return tcpPort;
+    }
+
+    /**
+     * Sets TCP port.
+     *
+     * @param tcpPort TCP port.
+     */
+    public void tcpPort(int tcpPort) {
+        this.tcpPort = tcpPort;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof HadoopProcessDescriptor))
+            return false;
+
+        HadoopProcessDescriptor that = (HadoopProcessDescriptor)o;
+
+        return parentNodeId.equals(that.parentNodeId) && procId.equals(that.procId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int result = parentNodeId.hashCode();
+
+        result = 31 * result + procId.hashCode();
+
+        return result;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopProcessDescriptor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessStartedAck.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessStartedAck.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessStartedAck.java
new file mode 100644
index 0000000..49ff4bf
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopProcessStartedAck.java
@@ -0,0 +1,46 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Process started message.
+ */
+public class HadoopProcessStartedAck implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopProcessStartedAck.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskExecutionRequest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskExecutionRequest.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskExecutionRequest.java
new file mode 100644
index 0000000..edf1840
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskExecutionRequest.java
@@ -0,0 +1,110 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Message sent from node to child process to start task(s) execution.
+ */
+public class HadoopTaskExecutionRequest implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Job ID. */
+    @GridToStringInclude
+    private GridHadoopJobId jobId;
+
+    /** Job info. */
+    @GridToStringInclude
+    private GridHadoopJobInfo jobInfo;
+
+    /** Mappers. */
+    @GridToStringInclude
+    private Collection<GridHadoopTaskInfo> tasks;
+
+    /**
+     * @return Job ID.
+     */
+    public GridHadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @param jobId Job ID.
+     */
+    public void jobId(GridHadoopJobId jobId) {
+        this.jobId = jobId;
+    }
+
+    /**
+     * @return Jon info.
+     */
+    public GridHadoopJobInfo jobInfo() {
+        return jobInfo;
+    }
+
+    /**
+     * @param jobInfo Job info.
+     */
+    public void jobInfo(GridHadoopJobInfo jobInfo) {
+        this.jobInfo = jobInfo;
+    }
+
+    /**
+     * @return Tasks.
+     */
+    public Collection<GridHadoopTaskInfo> tasks() {
+        return tasks;
+    }
+
+    /**
+     * @param tasks Tasks.
+     */
+    public void tasks(Collection<GridHadoopTaskInfo> tasks) {
+        this.tasks = tasks;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopTaskExecutionRequest.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+
+        out.writeObject(jobInfo);
+        U.writeCollection(out, tasks);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new GridHadoopJobId();
+        jobId.readExternal(in);
+
+        jobInfo = (GridHadoopJobInfo)in.readObject();
+        tasks = U.readCollection(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskFinishedMessage.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskFinishedMessage.java
new file mode 100644
index 0000000..a516f6b
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopTaskFinishedMessage.java
@@ -0,0 +1,92 @@
+/*
+ * 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.taskexecutor.external;
+
+import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.message.*;
+import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+
+/**
+ * Task finished message. Sent when local task finishes execution.
+ */
+public class HadoopTaskFinishedMessage implements HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Finished task info. */
+    private GridHadoopTaskInfo taskInfo;
+
+    /** Task finish status. */
+    private HadoopTaskStatus status;
+
+    /**
+     * Constructor required by {@link Externalizable}.
+     */
+    public HadoopTaskFinishedMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param taskInfo Finished task info.
+     * @param status Task finish status.
+     */
+    public HadoopTaskFinishedMessage(GridHadoopTaskInfo taskInfo, HadoopTaskStatus status) {
+        assert taskInfo != null;
+        assert status != null;
+
+        this.taskInfo = taskInfo;
+        this.status = status;
+    }
+
+    /**
+     * @return Finished task info.
+     */
+    public GridHadoopTaskInfo taskInfo() {
+        return taskInfo;
+    }
+
+    /**
+     * @return Task finish status.
+     */
+    public HadoopTaskStatus status() {
+        return status;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopTaskFinishedMessage.class, this);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        taskInfo.writeExternal(out);
+        status.writeExternal(out);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        taskInfo = new GridHadoopTaskInfo();
+        taskInfo.readExternal(in);
+
+        status = new HadoopTaskStatus();
+        status.readExternal(in);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
index 2d00222..21552e2 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
@@ -42,13 +42,13 @@ import static org.apache.ignite.internal.processors.hadoop.GridHadoopTaskType.*;
 @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
 public class GridHadoopChildProcessRunner {
     /** Node process descriptor. */
-    private GridHadoopProcessDescriptor nodeDesc;
+    private HadoopProcessDescriptor nodeDesc;
 
     /** Message processing executor service. */
     private ExecutorService msgExecSvc;
 
     /** Task executor service. */
-    private GridHadoopExecutorService execSvc;
+    private HadoopExecutorService execSvc;
 
     /** */
     protected GridUnsafeMemory mem = new GridUnsafeMemory(0);
@@ -75,7 +75,7 @@ public class GridHadoopChildProcessRunner {
     private final AtomicInteger pendingTasks = new AtomicInteger();
 
     /** Shuffle job. */
-    private GridHadoopShuffleJob<GridHadoopProcessDescriptor> shuffleJob;
+    private HadoopShuffleJob<HadoopProcessDescriptor> shuffleJob;
 
     /** Concurrent mappers. */
     private int concMappers;
@@ -86,7 +86,7 @@ public class GridHadoopChildProcessRunner {
     /**
      * Starts child process runner.
      */
-    public void start(GridHadoopExternalCommunication comm, GridHadoopProcessDescriptor nodeDesc,
+    public void start(GridHadoopExternalCommunication comm, HadoopProcessDescriptor nodeDesc,
         ExecutorService msgExecSvc, IgniteLogger parentLog)
         throws IgniteCheckedException {
         this.comm = comm;
@@ -99,7 +99,7 @@ public class GridHadoopChildProcessRunner {
         startTime = U.currentTimeMillis();
 
         // At this point node knows that this process has started.
-        comm.sendMessage(this.nodeDesc, new GridHadoopProcessStartedAck());
+        comm.sendMessage(this.nodeDesc, new HadoopProcessStartedAck());
     }
 
     /**
@@ -107,7 +107,7 @@ public class GridHadoopChildProcessRunner {
      *
      * @param req Initialization request.
      */
-    private void prepareProcess(GridHadoopPrepareForJobRequest req) {
+    private void prepareProcess(HadoopPrepareForJobRequest req) {
         if (initGuard.compareAndSet(false, true)) {
             try {
                 if (log.isDebugEnabled())
@@ -119,7 +119,7 @@ public class GridHadoopChildProcessRunner {
 
                 job.initialize(true, nodeDesc.processId());
 
-                shuffleJob = new GridHadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
+                shuffleJob = new HadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
                     req.totalReducerCount(), req.localReducers());
 
                 initializeExecutors(req);
@@ -143,7 +143,7 @@ public class GridHadoopChildProcessRunner {
     /**
      * @param req Task execution request.
      */
-    private void runTasks(final GridHadoopTaskExecutionRequest req) {
+    private void runTasks(final HadoopTaskExecutionRequest req) {
         if (!initFut.isDone() && log.isDebugEnabled())
             log.debug("Will wait for process initialization future completion: " + req);
 
@@ -175,7 +175,7 @@ public class GridHadoopChildProcessRunner {
                             log.debug("Submitted task for external execution: " + taskInfo);
 
                         execSvc.submit(new GridHadoopRunnableTask(log, job, mem, taskInfo, nodeDesc.parentNodeId()) {
-                            @Override protected void onTaskFinished(GridHadoopTaskStatus status) {
+                            @Override protected void onTaskFinished(HadoopTaskStatus status) {
                                 onTaskFinished0(this, status);
                             }
 
@@ -193,7 +193,7 @@ public class GridHadoopChildProcessRunner {
                 }
                 catch (IgniteCheckedException e) {
                     for (GridHadoopTaskInfo info : req.tasks())
-                        notifyTaskFinished(info, new GridHadoopTaskStatus(GridHadoopTaskState.FAILED, e), false);
+                        notifyTaskFinished(info, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
                 }
             }
         });
@@ -204,13 +204,13 @@ public class GridHadoopChildProcessRunner {
      *
      * @param req Init child process request.
      */
-    private void initializeExecutors(GridHadoopPrepareForJobRequest req) {
+    private void initializeExecutors(HadoopPrepareForJobRequest req) {
         int cpus = Runtime.getRuntime().availableProcessors();
 //
 //        concMappers = get(req.jobInfo(), EXTERNAL_CONCURRENT_MAPPERS, cpus);
 //        concReducers = get(req.jobInfo(), EXTERNAL_CONCURRENT_REDUCERS, cpus);
 
-        execSvc = new GridHadoopExecutorService(log, "", cpus * 2, 1024);
+        execSvc = new HadoopExecutorService(log, "", cpus * 2, 1024);
     }
 
     /**
@@ -218,7 +218,7 @@ public class GridHadoopChildProcessRunner {
      *
      * @param req Update request.
      */
-    private void updateTasks(final GridHadoopJobInfoUpdateRequest req) {
+    private void updateTasks(final HadoopJobInfoUpdateRequest req) {
         initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
             @Override public void apply(IgniteInternalFuture<?> gridFut) {
                 assert initGuard.get();
@@ -228,9 +228,9 @@ public class GridHadoopChildProcessRunner {
                 if (req.reducersAddresses() != null) {
                     if (shuffleJob.initializeReduceAddresses(req.reducersAddresses())) {
                         shuffleJob.startSending("external",
-                            new IgniteInClosure2X<GridHadoopProcessDescriptor, GridHadoopShuffleMessage>() {
-                                @Override public void applyx(GridHadoopProcessDescriptor dest,
-                                    GridHadoopShuffleMessage msg) throws IgniteCheckedException {
+                            new IgniteInClosure2X<HadoopProcessDescriptor, HadoopShuffleMessage>() {
+                                @Override public void applyx(HadoopProcessDescriptor dest,
+                                    HadoopShuffleMessage msg) throws IgniteCheckedException {
                                     comm.sendMessage(dest, msg);
                                 }
                             });
@@ -264,7 +264,7 @@ public class GridHadoopChildProcessRunner {
      * @param run Finished task runnable.
      * @param status Task status.
      */
-    private void onTaskFinished0(GridHadoopRunnableTask run, GridHadoopTaskStatus status) {
+    private void onTaskFinished0(GridHadoopRunnableTask run, HadoopTaskStatus status) {
         GridHadoopTaskInfo info = run.taskInfo();
 
         int pendingTasks0 = pendingTasks.decrementAndGet();
@@ -286,10 +286,10 @@ public class GridHadoopChildProcessRunner {
      * @param taskInfo Finished task info.
      * @param status Task status.
      */
-    private void notifyTaskFinished(final GridHadoopTaskInfo taskInfo, final GridHadoopTaskStatus status,
+    private void notifyTaskFinished(final GridHadoopTaskInfo taskInfo, final HadoopTaskStatus status,
         boolean flush) {
 
-        final GridHadoopTaskState state = status.state();
+        final HadoopTaskState state = status.state();
         final Throwable err = status.failCause();
 
         if (!flush) {
@@ -298,7 +298,7 @@ public class GridHadoopChildProcessRunner {
                     log.debug("Sending notification to parent node [taskInfo=" + taskInfo + ", state=" + state +
                         ", err=" + err + ']');
 
-                comm.sendMessage(nodeDesc, new GridHadoopTaskFinishedMessage(taskInfo, status));
+                comm.sendMessage(nodeDesc, new HadoopTaskFinishedMessage(taskInfo, status));
             }
             catch (IgniteCheckedException e) {
                 log.error("Failed to send message to parent node (will terminate child process).", e);
@@ -335,7 +335,7 @@ public class GridHadoopChildProcessRunner {
                                 ", state=" + state + ", err=" + err + ']', e);
 
                             notifyTaskFinished(taskInfo,
-                                new GridHadoopTaskStatus(GridHadoopTaskState.FAILED, e), false);
+                                new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
                         }
                     }
                 });
@@ -344,7 +344,7 @@ public class GridHadoopChildProcessRunner {
                 log.error("Failed to flush shuffle messages (will fail the task) [taskInfo=" + taskInfo +
                     ", state=" + state + ", err=" + err + ']', e);
 
-                notifyTaskFinished(taskInfo, new GridHadoopTaskStatus(GridHadoopTaskState.FAILED, e), false);
+                notifyTaskFinished(taskInfo, new HadoopTaskStatus(HadoopTaskState.FAILED, e), false);
             }
         }
     }
@@ -356,7 +356,7 @@ public class GridHadoopChildProcessRunner {
      * @param msg Received message.
      * @return {@code True} if received from parent node.
      */
-    private boolean validateNodeMessage(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
+    private boolean validateNodeMessage(HadoopProcessDescriptor desc, HadoopMessage msg) {
         if (!nodeDesc.processId().equals(desc.processId())) {
             log.warning("Received process control request from unknown process (will ignore) [desc=" + desc +
                 ", msg=" + msg + ']');
@@ -379,31 +379,31 @@ public class GridHadoopChildProcessRunner {
      */
     private class MessageListener implements GridHadoopMessageListener {
         /** {@inheritDoc} */
-        @Override public void onMessageReceived(final GridHadoopProcessDescriptor desc, final GridHadoopMessage msg) {
-            if (msg instanceof GridHadoopTaskExecutionRequest) {
+        @Override public void onMessageReceived(final HadoopProcessDescriptor desc, final HadoopMessage msg) {
+            if (msg instanceof HadoopTaskExecutionRequest) {
                 if (validateNodeMessage(desc, msg))
-                    runTasks((GridHadoopTaskExecutionRequest)msg);
+                    runTasks((HadoopTaskExecutionRequest)msg);
             }
-            else if (msg instanceof GridHadoopJobInfoUpdateRequest) {
+            else if (msg instanceof HadoopJobInfoUpdateRequest) {
                 if (validateNodeMessage(desc, msg))
-                    updateTasks((GridHadoopJobInfoUpdateRequest)msg);
+                    updateTasks((HadoopJobInfoUpdateRequest)msg);
             }
-            else if (msg instanceof GridHadoopPrepareForJobRequest) {
+            else if (msg instanceof HadoopPrepareForJobRequest) {
                 if (validateNodeMessage(desc, msg))
-                    prepareProcess((GridHadoopPrepareForJobRequest)msg);
+                    prepareProcess((HadoopPrepareForJobRequest)msg);
             }
-            else if (msg instanceof GridHadoopShuffleMessage) {
+            else if (msg instanceof HadoopShuffleMessage) {
                 if (log.isTraceEnabled())
                     log.trace("Received shuffle message [desc=" + desc + ", msg=" + msg + ']');
 
                 initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> f) {
                         try {
-                            GridHadoopShuffleMessage m = (GridHadoopShuffleMessage)msg;
+                            HadoopShuffleMessage m = (HadoopShuffleMessage)msg;
 
                             shuffleJob.onShuffleMessage(m);
 
-                            comm.sendMessage(desc, new GridHadoopShuffleAck(m.id(), m.jobId()));
+                            comm.sendMessage(desc, new HadoopShuffleAck(m.id(), m.jobId()));
                         }
                         catch (IgniteCheckedException e) {
                             U.error(log, "Failed to process hadoop shuffle message [desc=" + desc + ", msg=" + msg + ']', e);
@@ -411,18 +411,18 @@ public class GridHadoopChildProcessRunner {
                     }
                 });
             }
-            else if (msg instanceof GridHadoopShuffleAck) {
+            else if (msg instanceof HadoopShuffleAck) {
                 if (log.isTraceEnabled())
                     log.trace("Received shuffle ack [desc=" + desc + ", msg=" + msg + ']');
 
-                shuffleJob.onShuffleAck((GridHadoopShuffleAck)msg);
+                shuffleJob.onShuffleAck((HadoopShuffleAck)msg);
             }
             else
                 log.warning("Unknown message received (will ignore) [desc=" + desc + ", msg=" + msg + ']');
         }
 
         /** {@inheritDoc} */
-        @Override public void onConnectionLost(GridHadoopProcessDescriptor desc) {
+        @Override public void onConnectionLost(HadoopProcessDescriptor desc) {
             if (log.isDebugEnabled())
                 log.debug("Lost connection with remote process: " + desc);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
index 5aeeeee..1216c9a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopExternalProcessStarter.java
@@ -98,7 +98,7 @@ public class GridHadoopExternalProcessStarter {
 
         comm.start();
 
-        GridHadoopProcessDescriptor nodeDesc = new GridHadoopProcessDescriptor(args.nodeId, args.parentProcId);
+        HadoopProcessDescriptor nodeDesc = new HadoopProcessDescriptor(args.nodeId, args.parentProcId);
         nodeDesc.address(args.addr);
         nodeDesc.tcpPort(args.tcpPort);
         nodeDesc.sharedMemoryPort(args.shmemPort);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
index b375b55..f4eb41a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopCommunicationClient.java
@@ -68,5 +68,5 @@ public interface GridHadoopCommunicationClient {
      * @param msg Message to send.
      * @throws IgniteCheckedException If failed.
      */
-    public void sendMessage(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) throws IgniteCheckedException;
+    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
index f5ddced..937e245 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopExternalCommunication.java
@@ -87,10 +87,10 @@ public class GridHadoopExternalCommunication {
     public static final boolean DFLT_TCP_NODELAY = true;
 
     /** Server listener. */
-    private final GridNioServerListener<GridHadoopMessage> srvLsnr =
-        new GridNioServerListenerAdapter<GridHadoopMessage>() {
+    private final GridNioServerListener<HadoopMessage> srvLsnr =
+        new GridNioServerListenerAdapter<HadoopMessage>() {
             @Override public void onConnected(GridNioSession ses) {
-                GridHadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
 
                 assert desc != null : "Received connected notification without finished handshake: " + ses;
             }
@@ -103,7 +103,7 @@ public class GridHadoopExternalCommunication {
                 if (e != null)
                     U.error(log, "Session disconnected due to exception: " + ses, e);
 
-                GridHadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+                HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
 
                 if (desc != null) {
                     GridHadoopCommunicationClient rmv = clients.remove(desc.processId());
@@ -120,8 +120,8 @@ public class GridHadoopExternalCommunication {
             }
 
             /** {@inheritDoc} */
-            @Override public void onMessage(GridNioSession ses, GridHadoopMessage msg) {
-                notifyListener(ses.<GridHadoopProcessDescriptor>meta(PROCESS_META), msg);
+            @Override public void onMessage(GridNioSession ses, HadoopMessage msg) {
+                notifyListener(ses.<HadoopProcessDescriptor>meta(PROCESS_META), msg);
 
                 if (msgQueueLimit > 0) {
                     GridNioMessageTracker tracker = ses.meta(TRACKER_META);
@@ -137,7 +137,7 @@ public class GridHadoopExternalCommunication {
     private IgniteLogger log;
 
     /** Local process descriptor. */
-    private GridHadoopProcessDescriptor locProcDesc;
+    private HadoopProcessDescriptor locProcDesc;
 
     /** Marshaller. */
     private Marshaller marsh;
@@ -183,7 +183,7 @@ public class GridHadoopExternalCommunication {
     private int msgQueueLimit = DFLT_MSG_QUEUE_LIMIT;
 
     /** NIO server. */
-    private GridNioServer<GridHadoopMessage> nioSrvr;
+    private GridNioServer<HadoopMessage> nioSrvr;
 
     /** Shared memory server. */
     private IpcSharedMemoryServerEndpoint shmemSrv;
@@ -234,7 +234,7 @@ public class GridHadoopExternalCommunication {
         ExecutorService execSvc,
         String gridName
     ) {
-        locProcDesc = new GridHadoopProcessDescriptor(parentNodeId, procId);
+        locProcDesc = new HadoopProcessDescriptor(parentNodeId, procId);
 
         this.marsh = marsh;
         this.log = log.getLogger(GridHadoopExternalCommunication.class);
@@ -563,7 +563,7 @@ public class GridHadoopExternalCommunication {
      *
      * @return Local process descriptor.
      */
-    public GridHadoopProcessDescriptor localProcessDescriptor() {
+    public HadoopProcessDescriptor localProcessDescriptor() {
         return locProcDesc;
     }
 
@@ -587,7 +587,7 @@ public class GridHadoopExternalCommunication {
      * @return Server instance.
      * @throws IgniteCheckedException Thrown if it's not possible to create server.
      */
-    private GridNioServer<GridHadoopMessage> resetNioServer() throws IgniteCheckedException {
+    private GridNioServer<HadoopMessage> resetNioServer() throws IgniteCheckedException {
         if (boundTcpPort >= 0)
             throw new IgniteCheckedException("Tcp NIO server was already created on port " + boundTcpPort);
 
@@ -596,8 +596,8 @@ public class GridHadoopExternalCommunication {
         // If configured TCP port is busy, find first available in range.
         for (int port = locPort; port < locPort + locPortRange; port++) {
             try {
-                GridNioServer<GridHadoopMessage> srvr =
-                    GridNioServer.<GridHadoopMessage>builder()
+                GridNioServer<HadoopMessage> srvr =
+                    GridNioServer.<HadoopMessage>builder()
                         .address(locHost)
                         .port(port)
                         .listener(srvLsnr)
@@ -722,7 +722,7 @@ public class GridHadoopExternalCommunication {
      * @param msg
      * @throws IgniteCheckedException
      */
-    public void sendMessage(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) throws
+    public void sendMessage(HadoopProcessDescriptor desc, HadoopMessage msg) throws
         IgniteCheckedException {
         assert desc != null;
         assert msg != null;
@@ -761,7 +761,7 @@ public class GridHadoopExternalCommunication {
      * @return The existing or just created client.
      * @throws IgniteCheckedException Thrown if any exception occurs.
      */
-    private GridHadoopCommunicationClient reserveClient(GridHadoopProcessDescriptor desc) throws IgniteCheckedException {
+    private GridHadoopCommunicationClient reserveClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
         assert desc != null;
 
         UUID procId = desc.processId();
@@ -806,7 +806,7 @@ public class GridHadoopExternalCommunication {
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridHadoopCommunicationClient createNioClient(GridHadoopProcessDescriptor desc)
+    @Nullable protected GridHadoopCommunicationClient createNioClient(HadoopProcessDescriptor desc)
         throws  IgniteCheckedException {
         assert desc != null;
 
@@ -837,7 +837,7 @@ public class GridHadoopExternalCommunication {
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridHadoopCommunicationClient createShmemClient(GridHadoopProcessDescriptor desc, int port)
+    @Nullable protected GridHadoopCommunicationClient createShmemClient(HadoopProcessDescriptor desc, int port)
         throws IgniteCheckedException {
         int attempt = 1;
 
@@ -929,7 +929,7 @@ public class GridHadoopExternalCommunication {
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridHadoopCommunicationClient createTcpClient(GridHadoopProcessDescriptor desc) throws IgniteCheckedException {
+    protected GridHadoopCommunicationClient createTcpClient(HadoopProcessDescriptor desc) throws IgniteCheckedException {
         String addr = desc.address();
 
         int port = desc.tcpPort();
@@ -1066,7 +1066,7 @@ public class GridHadoopExternalCommunication {
      * @param desc Sender process descriptor.
      * @param msg Communication message.
      */
-    protected void notifyListener(GridHadoopProcessDescriptor desc, GridHadoopMessage msg) {
+    protected void notifyListener(HadoopProcessDescriptor desc, HadoopMessage msg) {
         GridHadoopMessageListener lsnr = this.lsnr;
 
         if (lsnr != null)
@@ -1135,7 +1135,7 @@ public class GridHadoopExternalCommunication {
         private final IpcEndpoint endpoint;
 
         /** Adapter. */
-        private GridHadoopIpcToNioAdapter<GridHadoopMessage> adapter;
+        private GridHadoopIpcToNioAdapter<HadoopMessage> adapter;
 
         /**
          * @param endpoint Endpoint.
@@ -1279,7 +1279,7 @@ public class GridHadoopExternalCommunication {
 
         /** {@inheritDoc} */
         @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
-            GridHadoopProcessDescriptor desc = ses.meta(PROCESS_META);
+            HadoopProcessDescriptor desc = ses.meta(PROCESS_META);
 
             UUID rmtProcId = desc == null ? null : desc.processId();
 
@@ -1387,12 +1387,12 @@ public class GridHadoopExternalCommunication {
      * Process ID message.
      */
     @SuppressWarnings("PublicInnerClass")
-    public static class ProcessHandshakeMessage implements GridHadoopMessage {
+    public static class ProcessHandshakeMessage implements HadoopMessage {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** Node ID. */
-        private GridHadoopProcessDescriptor procDesc;
+        private HadoopProcessDescriptor procDesc;
 
         /** */
         public ProcessHandshakeMessage() {
@@ -1402,14 +1402,14 @@ public class GridHadoopExternalCommunication {
         /**
          * @param procDesc Process descriptor.
          */
-        private ProcessHandshakeMessage(GridHadoopProcessDescriptor procDesc) {
+        private ProcessHandshakeMessage(HadoopProcessDescriptor procDesc) {
             this.procDesc = procDesc;
         }
 
         /**
          * @return Process ID.
          */
-        public GridHadoopProcessDescriptor processDescriptor() {
+        public HadoopProcessDescriptor processDescriptor() {
             return procDesc;
         }
 
@@ -1420,7 +1420,7 @@ public class GridHadoopExternalCommunication {
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            procDesc = (GridHadoopProcessDescriptor)in.readObject();
+            procDesc = (HadoopProcessDescriptor)in.readObject();
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
index 2a25357..e9dfc92 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMarshallerFilter.java
@@ -55,7 +55,7 @@ public class GridHadoopMarshallerFilter extends GridNioFilterAdapter {
 
     /** {@inheritDoc} */
     @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        assert msg instanceof GridHadoopMessage : "Invalid message type: " + msg;
+        assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
 
         return proceedSessionWrite(ses, marshaller.marshal(msg));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/288709a1/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
index 219f4db..6010a8d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/GridHadoopMessageListener.java
@@ -28,12 +28,12 @@ public interface GridHadoopMessageListener {
      * @param desc Process descriptor.
      * @param msg Hadoop message.
      */
-    public void onMessageReceived(GridHadoopProcessDescriptor desc, GridHadoopMessage msg);
+    public void onMessageReceived(HadoopProcessDescriptor desc, HadoopMessage msg);
 
     /**
      * Called when connection to remote process was lost.
      *
      * @param desc Process descriptor.
      */
-    public void onConnectionLost(GridHadoopProcessDescriptor desc);
+    public void onConnectionLost(HadoopProcessDescriptor desc);
 }